You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@apex.apache.org by th...@apache.org on 2016/08/25 16:43:17 UTC

[1/6] apex-malhar git commit: Added Beam Examples and Implementations of Accumulation.

Repository: apex-malhar
Updated Branches:
  refs/heads/master 17f6c5523 -> dcca7752a


http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java
deleted file mode 100644
index 77a08a6..0000000
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java
+++ /dev/null
@@ -1,107 +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.apex.malhar.stream.api.impl.accumulation;
-
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.apex.malhar.lib.window.Accumulation;
-
-/**
- * TopN accumulation
- */
-public class TopN<T> implements Accumulation<T, List<T>, List<T>>
-{
-
-  int n;
-
-  Comparator<T> comparator;
-
-  public void setN(int n)
-  {
-    this.n = n;
-  }
-
-  public void setComparator(Comparator<T> comparator)
-  {
-    this.comparator = comparator;
-  }
-
-  @Override
-  public List<T> defaultAccumulatedValue()
-  {
-    return new LinkedList<>();
-  }
-
-  @Override
-  public List<T> accumulate(List<T> accumulatedValue, T input)
-  {
-    int k = 0;
-    for (T inMemory : accumulatedValue) {
-      if (comparator != null) {
-        if (comparator.compare(inMemory, input) < 0) {
-          break;
-        }
-      } else if (input instanceof Comparable) {
-        if (((Comparable<T>)input).compareTo(inMemory) > 0) {
-          break;
-        }
-      } else {
-        throw new RuntimeException("Tuple cannot be compared");
-      }
-      k++;
-    }
-    accumulatedValue.add(k, input);
-    if (accumulatedValue.size() > n) {
-      accumulatedValue.remove(accumulatedValue.get(accumulatedValue.size() - 1));
-    }
-    return accumulatedValue;
-  }
-
-  @Override
-  public List<T> merge(List<T> accumulatedValue1, List<T> accumulatedValue2)
-  {
-    accumulatedValue1.addAll(accumulatedValue2);
-    if (comparator != null) {
-      Collections.sort(accumulatedValue1, Collections.reverseOrder(comparator));
-    } else {
-      Collections.sort(accumulatedValue1, Collections.reverseOrder());
-    }
-    if (accumulatedValue1.size() > n) {
-      return accumulatedValue1.subList(0, n);
-    } else {
-      return accumulatedValue1;
-    }
-  }
-
-  @Override
-  public List<T> getOutput(List<T> accumulatedValue)
-  {
-    return accumulatedValue;
-  }
-
-  @Override
-  public List<T> getRetraction(List<T> accumulatedValue)
-  {
-    return new LinkedList<>();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/stream/src/test/resources/words/word.txt
----------------------------------------------------------------------
diff --git a/stream/src/test/resources/words/word.txt b/stream/src/test/resources/words/word.txt
new file mode 100644
index 0000000..a8e8c35
--- /dev/null
+++ b/stream/src/test/resources/words/word.txt
@@ -0,0 +1,2 @@
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error bye


[3/6] apex-malhar git commit: Added Beam Examples and Implementations of Accumulation.

Posted by th...@apache.org.
http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java
index 5d4c628..ecd71ae 100644
--- a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java
@@ -18,19 +18,33 @@
  */
 package org.apache.apex.malhar.stream.sample.cookbook;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
 import org.apache.apex.malhar.lib.window.Tuple;
 import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.lib.window.impl.accumulation.ReduceFn;
 import org.apache.apex.malhar.stream.api.ApexStream;
 import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.WindowedStream;
 import org.apache.apex.malhar.stream.api.function.Function;
 import org.apache.apex.malhar.stream.api.impl.StreamFactory;
-import org.apache.apex.malhar.stream.api.impl.accumulation.ReduceFn;
+
+import org.apache.hadoop.conf.Configuration;
 
 import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
 import com.datatorrent.api.DefaultOutputPort;
 import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.common.util.BaseOperator;
 import com.datatorrent.lib.util.KeyValPair;
 
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
 /**
  * An example that reads the public 'Shakespeare' data, and for each word in
  * the dataset that is over a given length, generates a string containing the
@@ -40,12 +54,13 @@ import com.datatorrent.lib.util.KeyValPair;
  * key-grouped Collection
  *
  */
-public class CombinePerKeyExamples
+@ApplicationAnnotation(name = "CombinePerKeyExamples")
+public class CombinePerKeyExamples implements StreamingApplication
 {
   // Use the shakespeare public BigQuery sample
   private static final String SHAKESPEARE_TABLE = "publicdata:samples.shakespeare";
   // We'll track words >= this word length across all plays in the table.
-  private static final int MIN_WORD_LENGTH = 9;
+  private static final int MIN_WORD_LENGTH = 0;
 
   /**
    * Examines each row in the input table. If the word is greater than or equal to MIN_WORD_LENGTH,
@@ -76,70 +91,59 @@ public class CombinePerKeyExamples
     @Override
     public SampleBean f(Tuple.WindowedTuple<KeyValPair<String, String>> input)
     {
-      return new SampleBean(input.getValue().getKey(), input.getValue().getValue(), null);
+      return new SampleBean(input.getValue().getKey(), input.getValue().getValue());
     }
   }
-
+  
+  /**
+   * A reduce function to concat two strings together.
+   */
+  public static class Concat extends ReduceFn<String>
+  {
+    @Override
+    public String reduce(String input1, String input2)
+    {
+      return input1 + ", " + input2;
+    }
+  }
+  
   /**
    * Reads the public 'Shakespeare' data, and for each word in the dataset
    * over a given length, generates a string containing the list of play names
    * in which that word appears.
    */
-  static class PlaysForWord
-      extends CompositeStreamTransform<SampleBean, SampleBean>
+  private static class PlaysForWord extends CompositeStreamTransform<ApexStream<SampleBean>, WindowedStream<SampleBean>>
   {
-
+    
     @Override
-    public ApexStream<SampleBean> compose(ApexStream<SampleBean> inputStream)
+    public WindowedStream<SampleBean> compose(ApexStream<SampleBean> inputStream)
     {
-      // fix this later
-      return inputStream.map(new ExtractLargeWordsFn())
-          .window(new WindowOption.GlobalWindow())
-          .reduceByKey(new ReduceFn<String>()
-          {
-            @Override
-            public String defaultAccumulatedValue()
-            {
-              return "";
-            }
-
-            @Override
-            public String accumulate(String accumulatedValue, String input)
-            {
-              return accumulatedValue + "," + input;
-            }
-
-            @Override
-            public String merge(String accumulatedValue1, String accumulatedValue2)
-            {
-              return accumulatedValue1 + "," + accumulatedValue2;
-            }
-
-            @Override
-            public String getOutput(String accumulatedValue)
-            {
-              return accumulatedValue;
-            }
-
-            @Override
-            public String getRetraction(String value)
-            {
-              return value;
-            }
-          }, new Function.MapFunction<KeyValPair<String, String>, Tuple<KeyValPair<String, String>>>()
-
+      return inputStream
+          // Extract words from the input SampleBeam stream.
+          .map(new ExtractLargeWordsFn(), name("ExtractLargeWordsFn"))
+          
+          // Apply window and trigger option to the streams.
+          .window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
+        
+          // Apply reduceByKey transformation to concat the names of all the plays that a word has appeared in together.
+          .reduceByKey(new Concat(), new Function.ToKeyValue<KeyValPair<String,String>, String, String>()
           {
             @Override
             public Tuple<KeyValPair<String, String>> f(KeyValPair<String, String> input)
             {
-              return null;
+              return new Tuple.PlainTuple<KeyValPair<String, String>>(input);
             }
-          })
-          .map(new FormatShakespeareOutputFn());
+          }, name("Concat"))
+        
+          // Format the output back to a SampleBeam object.
+          .map(new FormatShakespeareOutputFn(), name("FormatShakespeareOutputFn"));
     }
   }
-
-
+  
+  
+  /**
+   * A Java Beam class that contains information about a word appears in a corpus written by Shakespeare.
+   */
   public static class SampleBean
   {
 
@@ -148,17 +152,20 @@ public class CombinePerKeyExamples
 
     }
 
-    public SampleBean(String word, String all_plays, String corpus)
+    public SampleBean(String word, String corpus)
     {
       this.word = word;
-      this.all_plays = all_plays;
       this.corpus = corpus;
     }
-
+  
+    @Override
+    public String toString()
+    {
+      return this.word + " : "  + this.corpus;
+    }
+  
     private String word;
 
-    private String all_plays;
-
     private String corpus;
 
     public void setWord(String word)
@@ -180,58 +187,87 @@ public class CombinePerKeyExamples
     {
       return corpus;
     }
-
-    public void setAll_plays(String all_plays)
-    {
-      this.all_plays = all_plays;
-    }
-
-    public String getAll_plays()
-    {
-      return all_plays;
-    }
   }
-
-  public static class SampleInput implements InputOperator
+  
+  /**
+   * A dummy info generator to generate {@link SampleBean} objects to mimic reading from real 'Shakespeare'
+   * data.
+   */
+  public static class SampleInput extends BaseOperator implements InputOperator
   {
 
     public final transient DefaultOutputPort<SampleBean> beanOutput = new DefaultOutputPort();
-
-    @Override
-    public void emitTuples()
+    private String[] words = new String[]{"A", "B", "C", "D", "E", "F", "G"};
+    private String[] corpuses = new String[]{"1", "2", "3", "4", "5", "6", "7", "8"};
+    private static int i;
+  
+    public static int getI()
     {
-
+      return i;
     }
-
+  
     @Override
-    public void beginWindow(long l)
+    public void setup(Context.OperatorContext context)
     {
-
+      super.setup(context);
+      i = 0;
     }
-
+  
     @Override
-    public void endWindow()
+    public void emitTuples()
     {
-
+      while (i < 1) {
+        for (String word : words) {
+          for (String corpus : corpuses) {
+            beanOutput.emit(new SampleBean(word, corpus));
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException e) {
+              // Ignore it
+            }
+          }
+        }
+        i++;
+      }
+    
     }
-
+  }
+  
+  public static class Collector extends BaseOperator
+  {
+    static List<SampleBean> result;
+  
     @Override
     public void setup(Context.OperatorContext context)
     {
-
+      result = new ArrayList<>();
     }
-
-    @Override
-    public void teardown()
+  
+    public final transient DefaultInputPort<SampleBean> input = new DefaultInputPort<SampleBean>()
     {
-
-    }
+      @Override
+      public void process(SampleBean tuple)
+      {
+        result.add(tuple);
+      }
+    };
   }
-
-
-  public static void main(String[] args) throws Exception
+  
+  /**
+   * Populate dag using High-Level API.
+   * @param dag
+   * @param conf
+   */
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
   {
     SampleInput input = new SampleInput();
-    StreamFactory.fromInput(input, input.beanOutput).addCompositeStreams(new PlaysForWord());
+    Collector collector = new Collector();
+    StreamFactory.fromInput(input, input.beanOutput, name("input"))
+      .addCompositeStreams(new PlaysForWord())
+      .print()
+      .endWith(collector, collector.input, name("Collector"))
+      .populateDag(dag);
+    
   }
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExample.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExample.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExample.java
new file mode 100644
index 0000000..53426f3
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExample.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.cookbook;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.lib.window.impl.accumulation.RemoveDuplicates;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * Beam DeDupExample.
+ */
+@ApplicationAnnotation(name = "DeDupExample")
+public class DeDupExample implements StreamingApplication
+{
+  
+  public static class Collector extends BaseOperator
+  {
+    private static Tuple.WindowedTuple<List<String>> result;
+    private static boolean done = false;
+  
+    public static Tuple.WindowedTuple<List<String>> getResult()
+    {
+      return result;
+    }
+  
+    public static boolean isDone()
+    {
+      return done;
+    }
+  
+    @Override
+    public void setup(Context.OperatorContext context)
+    {
+      super.setup(context);
+      result = new Tuple.WindowedTuple<>();
+      done = false;
+    }
+  
+    public transient DefaultInputPort<Tuple.WindowedTuple<List<String>>> input = new DefaultInputPort<Tuple.WindowedTuple<List<String>>>()
+    {
+      @Override
+      public void process(Tuple.WindowedTuple<List<String>> tuple)
+      {
+        result = tuple;
+        if (result.getValue().contains("bye")) {
+          done = true;
+        }
+      }
+    };
+  }
+    
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    Collector collector = new Collector();
+    
+    // Create a stream that reads from files in a local folder and output lines one by one to downstream.
+    ApexStream<String> stream = StreamFactory.fromFolder("./src/test/resources/wordcount", name("textInput"))
+      
+        // Extract all the words from the input line of text.
+        .flatMap(new Function.FlatMapFunction<String, String>()
+        {
+          @Override
+          public Iterable<String> f(String input)
+          {
+            return Arrays.asList(input.split("[\\p{Punct}\\s]+"));
+          }
+        }, name("ExtractWords"))
+      
+        // Change the words to lower case, also shutdown the app when the word "bye" is detected.
+        .map(new Function.MapFunction<String, String>()
+        {
+          @Override
+          public String f(String input)
+          {
+            return input.toLowerCase();
+          }
+        }, name("ToLowerCase"));
+    
+    // Apply window and trigger option.
+    stream.window(new WindowOption.GlobalWindow(),
+        new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(Duration.standardSeconds(1)))
+        
+        // Remove the duplicate words and print out the result.
+        .accumulate(new RemoveDuplicates<String>(), name("RemoveDuplicates")).print().endWith(collector, collector.input)
+    
+        .populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/InputPojo.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/InputPojo.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/InputPojo.java
new file mode 100644
index 0000000..3643eab
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/InputPojo.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.apex.malhar.stream.sample.cookbook;
+
+/**
+ * Tuple class for JDBC input of {@link MaxPerKeyExamples}.
+ */
+public class InputPojo extends Object
+{
+  private int month;
+  private int day;
+  private int year;
+  private double meanTemp;
+  
+  @Override
+  public String toString()
+  {
+    return "PojoEvent [month=" + getMonth() + ", day=" + getDay() + ", year=" + getYear() + ", meanTemp=" + getMeanTemp() + "]";
+  }
+  
+  public void setMonth(int month)
+  {
+    this.month = month;
+  }
+  
+  public int getMonth()
+  {
+    return this.month;
+  }
+  
+  public void setDay(int day)
+  {
+    this.day = day;
+  }
+  
+  public int getDay()
+  {
+    return day;
+  }
+  
+  public void setYear(int year)
+  {
+    this.year = year;
+  }
+  
+  public int getYear()
+  {
+    return year;
+  }
+  
+  public void setMeanTemp(double meanTemp)
+  {
+    this.meanTemp = meanTemp;
+  }
+  
+  public double getMeanTemp()
+  {
+    return meanTemp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamples.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamples.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamples.java
new file mode 100644
index 0000000..97b2696
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamples.java
@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.cookbook;
+
+import java.util.List;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.Window;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.lib.window.impl.accumulation.Max;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import static java.sql.Types.DOUBLE;
+import static java.sql.Types.INTEGER;
+
+import com.google.common.collect.Lists;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.lib.db.jdbc.JdbcFieldInfo;
+import com.datatorrent.lib.db.jdbc.JdbcPOJOInputOperator;
+import com.datatorrent.lib.db.jdbc.JdbcPOJOInsertOutputOperator;
+import com.datatorrent.lib.db.jdbc.JdbcStore;
+import com.datatorrent.lib.db.jdbc.JdbcTransactionalStore;
+import com.datatorrent.lib.util.FieldInfo;
+import com.datatorrent.lib.util.KeyValPair;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * MaxPerKeyExamples Application from Beam
+ */
+@ApplicationAnnotation(name = "MaxPerKeyExamples")
+public class MaxPerKeyExamples implements StreamingApplication
+{
+  
+  /**
+   *  A map function to extract the mean temperature from {@link InputPojo}.
+   */
+  public static class ExtractTempFn implements Function.MapFunction<InputPojo, KeyValPair<Integer, Double>>
+  {
+    @Override
+    public KeyValPair<Integer, Double> f(InputPojo row)
+    {
+      Integer month = row.getMonth();
+      Double meanTemp = row.getMeanTemp();
+      return new KeyValPair<Integer, Double>(month, meanTemp);
+    }
+  }
+  
+  
+  /**
+   * A map function to format output to {@link OutputPojo}.
+   */
+  public static class FormatMaxesFn implements Function.MapFunction<Tuple.WindowedTuple<KeyValPair<Integer, Double>>, OutputPojo>
+  {
+    @Override
+    public OutputPojo f(Tuple.WindowedTuple<KeyValPair<Integer, Double>> input)
+    {
+      OutputPojo row = new OutputPojo();
+      row.setMonth(input.getValue().getKey());
+      row.setMeanTemp(input.getValue().getValue());
+      return row;
+    }
+  }
+  
+  /**
+   * A composite transformation to perform three tasks:
+   * 1. extract the month and its mean temperature from input pojo.
+   * 2. find the maximum mean temperature for every month.
+   * 3. format the result to a output pojo object.
+   */
+  public static class MaxMeanTemp extends CompositeStreamTransform<WindowedStream<InputPojo>, WindowedStream<OutputPojo>>
+  {
+    @Override
+    public WindowedStream<OutputPojo> compose(WindowedStream<InputPojo> rows)
+    {
+      // InputPojo... => <month, meanTemp> ...
+      WindowedStream<KeyValPair<Integer, Double>> temps = rows.map(new ExtractTempFn(), name("ExtractTempFn"));
+      
+      // month, meanTemp... => <month, max mean temp>...
+      WindowedStream<Tuple.WindowedTuple<KeyValPair<Integer, Double>>> tempMaxes =
+          temps.accumulateByKey(new Max<Double>(),
+          new Function.ToKeyValue<KeyValPair<Integer, Double>, Integer, Double>()
+            {
+              @Override
+              public Tuple<KeyValPair<Integer, Double>> f(KeyValPair<Integer, Double> input)
+              {
+                return new Tuple.WindowedTuple<KeyValPair<Integer, Double>>(Window.GLOBAL_WINDOW, input);
+              }
+            }, name("MaxPerMonth"));
+      
+      // <month, max>... => OutputPojo...
+      WindowedStream<OutputPojo> results = tempMaxes.map(new FormatMaxesFn(), name("FormatMaxesFn"));
+      
+      return results;
+    }
+  }
+  
+  /**
+   * Method to set field info for {@link JdbcPOJOInputOperator}.
+   * @return
+   */
+  private List<FieldInfo> addInputFieldInfos()
+  {
+    List<FieldInfo> fieldInfos = Lists.newArrayList();
+    fieldInfos.add(new FieldInfo("MONTH", "month", FieldInfo.SupportType.INTEGER));
+    fieldInfos.add(new FieldInfo("DAY", "day", FieldInfo.SupportType.INTEGER));
+    fieldInfos.add(new FieldInfo("YEAR", "year", FieldInfo.SupportType.INTEGER));
+    fieldInfos.add(new FieldInfo("MEANTEMP", "meanTemp", FieldInfo.SupportType.DOUBLE));
+    return fieldInfos;
+  }
+  
+  /**
+   * Method to set field info for {@link JdbcPOJOInsertOutputOperator}.
+   * @return
+   */
+  private List<JdbcFieldInfo> addOutputFieldInfos()
+  {
+    List<JdbcFieldInfo> fieldInfos = Lists.newArrayList();
+    fieldInfos.add(new JdbcFieldInfo("MONTH", "month", JdbcFieldInfo.SupportType.INTEGER, INTEGER));
+    fieldInfos.add(new JdbcFieldInfo("MEANTEMP", "meanTemp", JdbcFieldInfo.SupportType.DOUBLE, DOUBLE));
+    return fieldInfos;
+  }
+  
+  
+  /**
+   * Populate the dag using High-Level API.
+   * @param dag
+   * @param conf
+   */
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    JdbcPOJOInputOperator jdbcInput = new JdbcPOJOInputOperator();
+    jdbcInput.setFieldInfos(addInputFieldInfos());
+  
+    JdbcStore store = new JdbcStore();
+    jdbcInput.setStore(store);
+  
+    JdbcPOJOInsertOutputOperator jdbcOutput = new JdbcPOJOInsertOutputOperator();
+    jdbcOutput.setFieldInfos(addOutputFieldInfos());
+    JdbcTransactionalStore outputStore = new JdbcTransactionalStore();
+    jdbcOutput.setStore(outputStore);
+    
+    // Create stream that reads from a Jdbc Input.
+    ApexStream<Object> stream = StreamFactory.fromInput(jdbcInput, jdbcInput.outputPort, name("jdbcInput"))
+      
+        // Apply window and trigger option to the stream.
+        .window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
+      
+        // Because Jdbc Input sends out a stream of Object, need to cast them to InputPojo.
+        .map(new Function.MapFunction<Object, InputPojo>()
+        {
+          @Override
+          public InputPojo f(Object input)
+          {
+            return (InputPojo)input;
+          }
+        }, name("ObjectToInputPojo"))
+      
+        // Plug in the composite transformation to the stream to calculate the maximum temperature for each month.
+        .addCompositeStreams(new MaxMeanTemp())
+      
+        // Cast the resulted OutputPojo to Object for Jdbc Output to consume.
+        .map(new Function.MapFunction<OutputPojo, Object>()
+        {
+          @Override
+          public Object f(OutputPojo input)
+          {
+            return (Object)input;
+          }
+        }, name("OutputPojoToObject"))
+      
+        // Output the result to Jdbc Output.
+        .endWith(jdbcOutput, jdbcOutput.input, name("jdbcOutput"));
+    
+    stream.populateDag(dag);
+  
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/OutputPojo.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/OutputPojo.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/OutputPojo.java
new file mode 100644
index 0000000..db2a09e
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/OutputPojo.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.apex.malhar.stream.sample.cookbook;
+
+/**
+ * OutputPojo Tuple Class for jdbcOutput of {@link MaxPerKeyExamples}.
+ */
+public class OutputPojo
+{
+  private int month;
+  private double meanTemp;
+  
+  @Override
+  public String toString()
+  {
+    return "PojoEvent [month=" + getMonth() + ", meanTemp=" + getMeanTemp() + "]";
+  }
+  
+  public void setMonth(int month)
+  {
+    this.month = month;
+  }
+  
+  public int getMonth()
+  {
+    return this.month;
+  }
+  
+  public void setMeanTemp(double meanTemp)
+  {
+    this.meanTemp = meanTemp;
+  }
+  
+  public double getMeanTemp()
+  {
+    return meanTemp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java
index 903f624..bf23e3a 100644
--- a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java
@@ -24,6 +24,7 @@ import java.util.Objects;
 import org.joda.time.Duration;
 
 import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
 import org.apache.apex.malhar.lib.window.WindowOption;
 import org.apache.apex.malhar.stream.api.ApexStream;
 import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
@@ -85,31 +86,31 @@ import com.datatorrent.lib.util.KeyValPair;
  * Note: When you start up your pipeline, you'll initially see results from 'late' data. Wait after
  * the window duration, until the first pane of non-late data has been emitted, to see more
  * interesting results.
- * {@code SELECT * FROM enter_table_name WHERE trigger_type = "default" ORDER BY window DESC}
+ * {@code SELECT * FROM enter_table_name WHERE triggerType = "default" ORDER BY window DESC}
  *
  * <p> To see the late data i.e. dropped by the default trigger,
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "withAllowedLateness" and
- * (timing = "LATE" or timing = "ON_TIME") and freeway = "5" ORDER BY window DESC, processing_time}
+ * {@code SELECT * FROM <enter_table_name> WHERE triggerType = "withAllowedLateness" and
+ * (timing = "LATE" or timing = "ON_TIME") and freeway = "5" ORDER BY window DESC, processingTime}
  *
  * <p>To see the the difference between accumulation mode and discarding mode,
  * {@code SELECT * FROM <enter_table_name> WHERE (timing = "LATE" or timing = "ON_TIME") AND
- * (trigger_type = "withAllowedLateness" or trigger_type = "sequential") and freeway = "5" ORDER BY
- * window DESC, processing_time}
+ * (triggerType = "withAllowedLateness" or triggerType = "sequential") and freeway = "5" ORDER BY
+ * window DESC, processingTime}
  *
  * <p> To see speculative results every minute,
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "speculative" and freeway = "5"
- * ORDER BY window DESC, processing_time}
+ * {@code SELECT * FROM <enter_table_name> WHERE triggerType = "speculative" and freeway = "5"
+ * ORDER BY window DESC, processingTime}
  *
  * <p> To see speculative results every five minutes after the end of the window
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "sequential" and timing != "EARLY"
- * and freeway = "5" ORDER BY window DESC, processing_time}
+ * {@code SELECT * FROM <enter_table_name> WHERE triggerType = "sequential" and timing != "EARLY"
+ * and freeway = "5" ORDER BY window DESC, processingTime}
  *
  * <p> To see the first and the last pane for a freeway in a window for all the trigger types,
  * {@code SELECT * FROM <enter_table_name> WHERE (isFirst = true or isLast = true) ORDER BY window}
  *
  * <p> To reduce the number of results for each query we can add additional where clauses.
  * For examples, To see the results of the default trigger,
- * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "default" AND freeway = "5" AND
+ * {@code SELECT * FROM <enter_table_name> WHERE triggerType = "default" AND freeway = "5" AND
  * window = "<enter_window_interval>"}
  *
  * <p> The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
@@ -135,7 +136,7 @@ public class TriggerExample
    * The example uses "freeway" as the key. Event time is the timestamp associated with the data
    * element and processing time is the time when the data element gets processed in the pipeline.
    * For freeway 5, suppose there are 10 elements in the [10:00:00, 10:30:00) window.
-   * Key (freeway) | Value (total_flow) | event time | processing time
+   * Key (freeway) | Value (totalFlow) | event time | processing time
    * 5             | 50                 | 10:00:03   | 10:00:47
    * 5             | 30                 | 10:01:00   | 10:01:03
    * 5             | 30                 | 10:02:00   | 11:07:00
@@ -157,7 +158,7 @@ public class TriggerExample
    * close at 10:44:59, when the watermark passes 10:30:00.
    */
   static class CalculateTotalFlow
-      extends CompositeStreamTransform<String, SampleBean>
+      extends CompositeStreamTransform<ApexStream<String>, WindowedStream<SampleBean>>
   {
     private int windowDuration;
 
@@ -167,7 +168,7 @@ public class TriggerExample
     }
 
     @Override
-    public ApexStream<SampleBean> compose(ApexStream<String> inputStream)
+    public WindowedStream<SampleBean> compose(ApexStream<String> inputStream)
     {
       // Concept #1: The default triggering behavior
       // By default Dataflow uses a trigger which fires when the watermark has passed the end of the
@@ -182,14 +183,14 @@ public class TriggerExample
 
       // The results for the example above with the default trigger and zero allowed lateness
       // would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // Key (freeway) | Value (totalFlow) | numberOfRecords | isFirst | isLast | timing
       // 5             | 260                | 6                 | true    | true   | ON_TIME
 
       // At 11:03:00 (processing time) the system watermark may have advanced to 10:54:00. As a
       // result, when the data record with event time 10:05:00 arrives at 11:03:00, it is considered
       // late, and dropped.
-
-      ApexStream<SampleBean> defaultTriggerResults = inputStream
+  
+      WindowedStream<SampleBean> defaultTriggerResults = inputStream
           .window(new WindowOption.TimeWindows(Duration.standardMinutes(windowDuration)),
           new TriggerOption().discardingFiredPanes())
           .addCompositeStreams(new TotalFlow("default"));
@@ -205,13 +206,13 @@ public class TriggerExample
 
       // The results for the example above with the default trigger and ONE_DAY allowed lateness
       // would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // Key (freeway) | Value (totalFlow) | numberOfRecords | isFirst | isLast | timing
       // 5             | 260                | 6                 | true    | false  | ON_TIME
       // 5             | 60                 | 1                 | false   | false  | LATE
       // 5             | 30                 | 1                 | false   | false  | LATE
       // 5             | 20                 | 1                 | false   | false  | LATE
       // 5             | 60                 | 1                 | false   | false  | LATE
-      ApexStream<SampleBean> withAllowedLatenessResults = inputStream
+      WindowedStream<SampleBean> withAllowedLatenessResults = inputStream
           .window(new WindowOption.TimeWindows(Duration.standardMinutes(windowDuration)),
           new TriggerOption().discardingFiredPanes(),
           Duration.standardDays(1))
@@ -226,7 +227,7 @@ public class TriggerExample
       // We also use accumulatingFiredPanes to build up the results across each pane firing.
 
       // The results for the example above for this trigger would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // Key (freeway) | Value (totalFlow) | numberOfRecords | isFirst | isLast | timing
       // 5             | 80                 | 2                 | true    | false  | EARLY
       // 5             | 100                | 3                 | false   | false  | EARLY
       // 5             | 260                | 6                 | false   | false  | EARLY
@@ -258,7 +259,7 @@ public class TriggerExample
       // Every pane produced will either be EARLY, ON_TIME or LATE.
 
       // The results for the example above for this trigger would be:
-      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // Key (freeway) | Value (totalFlow) | numberOfRecords | isFirst | isLast | timing
       // 5             | 80                 | 2                 | true    | false  | EARLY
       // 5             | 100                | 3                 | false   | false  | EARLY
       // 5             | 260                | 6                 | false   | false  | EARLY
@@ -267,7 +268,7 @@ public class TriggerExample
       // 5             | 430                | 10                | false   | false  | LATE
 
       // For more possibilities of how to build advanced triggers, see {@link Trigger}.
-      ApexStream<SampleBean> sequentialResults = inputStream
+      WindowedStream<SampleBean> sequentialResults = inputStream
           .window(new WindowOption.TimeWindows(Duration.standardMinutes(windowDuration)),
               // Speculative every ONE_MINUTE
           new TriggerOption().withEarlyFiringsAtEvery(Duration.standardMinutes(1))
@@ -293,7 +294,7 @@ public class TriggerExample
    * objects, to save to BigQuery.
    */
   static class TotalFlow extends
-      CompositeStreamTransform<String, SampleBean>
+      CompositeStreamTransform<WindowedStream<String>, WindowedStream<SampleBean>>
   {
     private String triggerType;
 
@@ -303,13 +304,10 @@ public class TriggerExample
     }
 
     @Override
-    public ApexStream<SampleBean> compose(ApexStream<String> inputStream)
+    public WindowedStream<SampleBean> compose(WindowedStream<String> inputStream)
     {
-      if (!(inputStream instanceof WindowedStream)) {
-        throw new RuntimeException("Not supported here");
-      }
-      WindowedStream<String> windowedStream = (WindowedStream<String>)inputStream;
-      ApexStream<KeyValPair<String, Iterable<Integer>>> flowPerFreeway = windowedStream
+  
+      WindowedStream<KeyValPair<String, Iterable<Integer>>> flowPerFreeway = inputStream
           .groupByKey(new ExtractFlowInfo());
 
       return flowPerFreeway
@@ -361,13 +359,13 @@ public class TriggerExample
     {
     }
 
-    private String trigger_type;
+    private String triggerType;
 
     private String freeway;
 
-    private int total_flow;
+    private int totalFlow;
 
-    private long number_of_records;
+    private long numberOfRecords;
 
     private String window;
 
@@ -377,9 +375,9 @@ public class TriggerExample
 
     private Date timing;
 
-    private Date event_time;
+    private Date eventTime;
 
-    private Date processing_time;
+    private Date processingTime;
 
     @Override
     public boolean equals(Object o)
@@ -391,50 +389,49 @@ public class TriggerExample
         return false;
       }
       SampleBean that = (SampleBean)o;
-      return total_flow == that.total_flow &&
-          number_of_records == that.number_of_records &&
+      return totalFlow == that.totalFlow &&
+          numberOfRecords == that.numberOfRecords &&
           isFirst == that.isFirst &&
           isLast == that.isLast &&
-          Objects.equals(trigger_type, that.trigger_type) &&
+          Objects.equals(triggerType, that.triggerType) &&
           Objects.equals(freeway, that.freeway) &&
           Objects.equals(window, that.window) &&
           Objects.equals(timing, that.timing) &&
-          Objects.equals(event_time, that.event_time) &&
-          Objects.equals(processing_time, that.processing_time);
+          Objects.equals(eventTime, that.eventTime) &&
+          Objects.equals(processingTime, that.processingTime);
     }
 
     @Override
     public int hashCode()
     {
       return Objects
-          .hash(trigger_type, freeway, total_flow, number_of_records, window, isFirst, isLast, timing, event_time,
-              processing_time);
+          .hash(triggerType, freeway, totalFlow, numberOfRecords, window, isFirst, isLast, timing, eventTime,
+            processingTime);
     }
 
-    public SampleBean(String trigger_type, String freeway, int total_flow, long number_of_records, String window,
-        boolean isFirst, boolean isLast, Date timing, Date event_time, Date processing_time)
+    public SampleBean(String triggerType, String freeway, int totalFlow, long numberOfRecords, String window, boolean isFirst, boolean isLast, Date timing, Date eventTime, Date processingTime)
     {
 
-      this.trigger_type = trigger_type;
+      this.triggerType = triggerType;
       this.freeway = freeway;
-      this.total_flow = total_flow;
-      this.number_of_records = number_of_records;
+      this.totalFlow = totalFlow;
+      this.numberOfRecords = numberOfRecords;
       this.window = window;
       this.isFirst = isFirst;
       this.isLast = isLast;
       this.timing = timing;
-      this.event_time = event_time;
-      this.processing_time = processing_time;
+      this.eventTime = eventTime;
+      this.processingTime = processingTime;
     }
 
-    public String getTrigger_type()
+    public String getTriggerType()
     {
-      return trigger_type;
+      return triggerType;
     }
 
-    public void setTrigger_type(String trigger_type)
+    public void setTriggerType(String triggerType)
     {
-      this.trigger_type = trigger_type;
+      this.triggerType = triggerType;
     }
 
     public String getFreeway()
@@ -447,24 +444,24 @@ public class TriggerExample
       this.freeway = freeway;
     }
 
-    public int getTotal_flow()
+    public int getTotalFlow()
     {
-      return total_flow;
+      return totalFlow;
     }
 
-    public void setTotal_flow(int total_flow)
+    public void setTotalFlow(int totalFlow)
     {
-      this.total_flow = total_flow;
+      this.totalFlow = totalFlow;
     }
 
-    public long getNumber_of_records()
+    public long getNumberOfRecords()
     {
-      return number_of_records;
+      return numberOfRecords;
     }
 
-    public void setNumber_of_records(long number_of_records)
+    public void setNumberOfRecords(long numberOfRecords)
     {
-      this.number_of_records = number_of_records;
+      this.numberOfRecords = numberOfRecords;
     }
 
     public String getWindow()
@@ -507,24 +504,24 @@ public class TriggerExample
       this.timing = timing;
     }
 
-    public Date getEvent_time()
+    public Date getEventTime()
     {
-      return event_time;
+      return eventTime;
     }
 
-    public void setEvent_time(Date event_time)
+    public void setEventTime(Date eventTime)
     {
-      this.event_time = event_time;
+      this.eventTime = eventTime;
     }
 
-    public Date getProcessing_time()
+    public Date getProcessingTime()
     {
-      return processing_time;
+      return processingTime;
     }
 
-    public void setProcessing_time(Date processing_time)
+    public void setProcessingTime(Date processingTime)
     {
-      this.processing_time = processing_time;
+      this.processingTime = processingTime;
     }
   }
 
@@ -532,10 +529,10 @@ public class TriggerExample
    * Extract the freeway and total flow in a reading.
    * Freeway is used as key since we are calculating the total flow for each freeway.
    */
-  static class ExtractFlowInfo implements Function.MapFunction<String, KeyValPair<String, Integer>>
+  static class ExtractFlowInfo implements Function.ToKeyValue<String, String, Integer>
   {
     @Override
-    public KeyValPair<String, Integer> f(String input)
+    public Tuple<KeyValPair<String, Integer>> f(String input)
     {
       String[] laneInfo = input.split(",");
       if (laneInfo[0].equals("timestamp")) {
@@ -553,7 +550,7 @@ public class TriggerExample
       if (totalFlow == null || totalFlow <= 0) {
         return null;
       }
-      return new KeyValPair<>(freeway, totalFlow);
+      return new Tuple.PlainTuple<>(new KeyValPair<>(freeway, totalFlow));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/resources/META-INF/properties.xml
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/resources/META-INF/properties.xml b/demos/highlevelapi/src/main/resources/META-INF/properties.xml
new file mode 100644
index 0000000..ead0460
--- /dev/null
+++ b/demos/highlevelapi/src/main/resources/META-INF/properties.xml
@@ -0,0 +1,141 @@
+<?xml version="1.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.
+
+-->
+<configuration>
+  <!-- 
+  <property>
+    <name>dt.application.{appName}.operator.{opName}.prop.{propName}</name>
+    <value>some-default-value (if value is not specified, it is required from the user or custom config when launching)</value>
+  </property>
+  -->
+
+  <!-- Properties for TwitterAutoComplete, please fill out all of them to make the application work -->
+  <property>
+    <name>dt.application.TwitterAutoComplete.operator.tweetSampler.consumerKey</name>
+    <value></value>
+  </property>
+  <property>
+    <name>dt.application.TwitterAutoComplete.operator.tweetSampler.consumerSecret</name>
+    <value></value>
+  </property>
+  <property>
+    <name>dt.application.TwitterAutoComplete.operator.tweetSampler.accessToken</name>
+    <value></value>
+  </property>
+  <property>
+    <name>dt.application.TwitterAutoComplete.operator.tweetSampler.accessTokenSecret</name>
+    <value></value>
+  </property>
+
+  <!-- Properties for StreamingWordExtract -->
+  <property>
+    <name>dt.application.StreamingWordExtract.operator.jdbcOutput.prop.store.userName</name>
+    <value>root</value>
+  </property>
+  <property>
+    <name>dt.application.StreamingWordExtract.operator.jdbcOutput.prop.store.password</name>
+    <value>password</value>
+  </property>
+  <property>
+    <name>dt.application.StreamingWordExtract.operator.jdbcOutput.prop.store.databaseDriver</name>
+    <value>org.hsqldb.jdbcDriver</value>
+  </property>
+  <property>
+    <name>dt.application.StreamingWordExtract.operator.jdbcOutput.prop.batchSize</name>
+    <value>5</value>
+  </property>
+  <property>
+    <name>dt.application.StreamingWordExtract.operator.jdbcOutput.port.input.attr.TUPLE_CLASS</name>
+    <value>org.apache.apex.malhar.stream.sample.complete.PojoEvent</value>
+  </property>
+  <property>
+    <name>dt.application.StreamingWordExtract.operator.jdbcOutput.prop.store.databaseUrl</name>
+    <value>jdbc:hsqldb:mem:test</value>
+  </property>
+  <property>
+    <name>dt.application.StreamingWordExtract.operator.jdbcOutput.prop.tablename</name>
+    <value>Test</value>
+  </property>
+
+  <!-- Properties for MaxPerKeyExamples -->
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.prop.store.userName</name>
+    <value>root</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.prop.store.password</name>
+    <value>password</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.prop.store.databaseDriver</name>
+    <value>org.hsqldb.jdbcDriver</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.prop.batchSize</name>
+    <value>5</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.port.outputPort.attr.TUPLE_CLASS</name>
+    <value>org.apache.apex.malhar.stream.sample.cookbook.InputPojo</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.prop.store.databaseUrl</name>
+    <value>jdbc:hsqldb:mem:test</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.prop.tableName</name>
+    <value>InputTable</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcInput.prop.query</name>
+    <value>SELECT * FROM InputTable;</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcOutput.prop.store.userName</name>
+    <value>root</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcOutput.prop.store.password</name>
+    <value>password</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcOutput.prop.store.databaseDriver</name>
+    <value>org.hsqldb.jdbcDriver</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcOutput.prop.batchSize</name>
+    <value>5</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcOutput.port.input.attr.TUPLE_CLASS</name>
+    <value>org.apache.apex.malhar.stream.sample.cookbook.OutputPojo</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcOutput.prop.store.databaseUrl</name>
+    <value>jdbc:hsqldb:mem:test</value>
+  </property>
+  <property>
+    <name>dt.application.MaxPerKeyExamples.operator.jdbcOutput.prop.tablename</name>
+    <value>OutputTable</value>
+  </property>
+
+</configuration>
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/MinimalWordCountTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/MinimalWordCountTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/MinimalWordCountTest.java
new file mode 100644
index 0000000..101953f
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/MinimalWordCountTest.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample;
+
+import java.util.concurrent.Callable;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Test for {@link MinimalWordCount}.
+ */
+public class MinimalWordCountTest
+{
+  @Test
+  public void MinimalWordCountTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+  
+    MinimalWordCount app = new MinimalWordCount();
+
+    lma.prepareDAG(app, conf);
+ 
+    LocalMode.Controller lc = lma.getController();
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return MinimalWordCount.Collector.isDone();
+      }
+    });
+    
+    lc.run(10000);
+  
+    Assert.assertTrue(MinimalWordCount.Collector.result.get("error") == 7);
+    Assert.assertTrue(MinimalWordCount.Collector.result.get("word") == 119);
+    Assert.assertTrue(MinimalWordCount.Collector.result.get("bye") == 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/WindowedWordCountTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/WindowedWordCountTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/WindowedWordCountTest.java
new file mode 100644
index 0000000..952356f
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/WindowedWordCountTest.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample;
+
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.lib.util.KeyValPair;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Testing the TwitterAutoComplete Application. In order to run this test, you need to create an app
+ * at https://apps.twitter.com, then generate your consumer and access keys and tokens, and set the following properties
+ * for the application before running it:
+ * Your application consumer key,
+ * Your application consumer secret,
+ * Your twitter access token, and
+ * Your twitter access token secret.
+ */
+public class WindowedWordCountTest
+{
+  @Test
+  public void WindowedWordCountTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    lma.prepareDAG(new WindowedWordCount(), conf);
+    LocalMode.Controller lc = lma.getController();
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return WindowedWordCount.TextInput.isDone();
+      }
+    });
+    
+    lc.run(60000);
+    
+    Assert.assertEquals(127, countSum(WindowedWordCount.Collector.getResult()));
+    Assert.assertEquals(28, countSumWord(WindowedWordCount.Collector.getResult(), "word2"));
+    Assert.assertEquals(7, countSumWord(WindowedWordCount.Collector.getResult(), "error"));
+    Assert.assertEquals(21, countSumWord(WindowedWordCount.Collector.getResult(), "word9"));
+    Assert.assertEquals(1, countSumWord(WindowedWordCount.Collector.getResult(), "bye"));
+  }
+  
+  public long countSum(Map<KeyValPair<Long, String>, Long> map)
+  {
+    long sum = 0;
+    for (long count : map.values()) {
+      sum += count;
+    }
+    return sum;
+  }
+  
+  public long countSumWord(Map<KeyValPair<Long, String>, Long> map, String word)
+  {
+    long sum = 0;
+    for (Map.Entry<KeyValPair<Long, String>, Long> entry : map.entrySet()) {
+      if (entry.getKey().getValue().equals(word)) {
+        sum += entry.getValue();
+      }
+    }
+    return sum;
+  }
+  
+}
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/AutoCompleteTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/AutoCompleteTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/AutoCompleteTest.java
new file mode 100644
index 0000000..dc236f9
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/AutoCompleteTest.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.apex.malhar.stream.sample.complete;
+
+import java.util.concurrent.Callable;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Testing the AutoComplete Application
+ */
+public class AutoCompleteTest
+{
+
+  @Test
+  public void AutoCompleteTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    lma.prepareDAG(new AutoComplete(), conf);
+    LocalMode.Controller lc = lma.getController();
+    
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return AutoComplete.TweetsInput.isDone();
+      }
+    });
+    
+    lc.run(200000);
+  
+    Assert.assertTrue(AutoComplete.Collector.getResult().containsKey("chi"));
+    Assert.assertTrue(AutoComplete.Collector.getResult().containsKey("china"));
+    Assert.assertEquals(2, AutoComplete.Collector.getResult().get("china").get(0).getCount());
+    Assert.assertEquals("China", AutoComplete.Collector.getResult().get("china").get(0).getValue());
+    Assert.assertEquals(2, AutoComplete.Collector.getResult().get("d").size());
+    Assert.assertEquals(3, AutoComplete.Collector.getResult().get("f").size());
+    Assert.assertTrue(AutoComplete.Collector.getResult().get("f").get(0).getCount() >= AutoComplete.Collector.getResult().get("f").get(1).getCount());
+    Assert.assertTrue(AutoComplete.Collector.getResult().get("f").get(1).getCount() >= AutoComplete.Collector.getResult().get("f").get(2).getCount());
+  
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtractTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtractTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtractTest.java
new file mode 100644
index 0000000..bf9b030
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtractTest.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.concurrent.Callable;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Throwables;
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.lib.db.jdbc.JdbcTransactionalStore;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Testing StreamingWordExtract application
+ */
+public class StreamingWordExtractTest
+{
+  private static final String TUPLE_CLASS = "org.apache.apex.malhar.stream.sample.complete.PojoEvent";
+  private static final String DB_DRIVER = "org.h2.Driver";
+  private static final String DB_URL = "jdbc:h2:~/test";
+  private static final String TABLE_NAME = "Test";
+  private static final String USER_NAME = "root";
+  private static final String PSW = "password";
+
+  @BeforeClass
+  public static void setup()
+  {
+    try {
+      Class.forName(DB_DRIVER).newInstance();
+      
+      Connection con = DriverManager.getConnection(DB_URL,USER_NAME,PSW);
+      Statement stmt = con.createStatement();
+      
+      String createMetaTable = "CREATE TABLE IF NOT EXISTS " + JdbcTransactionalStore.DEFAULT_META_TABLE + " ( "
+          + JdbcTransactionalStore.DEFAULT_APP_ID_COL + " VARCHAR(100) NOT NULL, "
+          + JdbcTransactionalStore.DEFAULT_OPERATOR_ID_COL + " INT NOT NULL, "
+          + JdbcTransactionalStore.DEFAULT_WINDOW_COL + " BIGINT NOT NULL, "
+          + "UNIQUE (" + JdbcTransactionalStore.DEFAULT_APP_ID_COL + ", "
+          + JdbcTransactionalStore.DEFAULT_OPERATOR_ID_COL + ", " + JdbcTransactionalStore.DEFAULT_WINDOW_COL + ") "
+          + ")";
+      stmt.executeUpdate(createMetaTable);
+      
+      String createTable = "CREATE TABLE IF NOT EXISTS " + TABLE_NAME
+          + "(STRINGVALUE VARCHAR(255))";
+      stmt.executeUpdate(createTable);
+      
+    } catch (Throwable e) {
+      throw Throwables.propagate(e);
+    }
+  }
+  
+  @After
+  public void cleanTable()
+  {
+    try {
+      Connection con = DriverManager.getConnection(DB_URL,USER_NAME,PSW);
+      Statement stmt = con.createStatement();
+      String dropTable = "drop table " + TABLE_NAME;
+      stmt.executeUpdate(dropTable);
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+  
+  public void setConfig(Configuration conf)
+  {
+    conf.set("dt.operator.jdbcOutput.prop.store.userName", USER_NAME);
+    conf.set("dt.operator.jdbcOutput.prop.store.password", PSW);
+    conf.set("dt.operator.jdbcOutput.prop.store.databaseDriver", DB_DRIVER);
+    conf.set("dt.operator.jdbcOutput.prop.batchSize", "5");
+    conf.set("dt.operator.jdbcOutput.port.input.attr.TUPLE_CLASS", TUPLE_CLASS);
+    conf.set("dt.operator.jdbcOutput.prop.store.databaseUrl", DB_URL);
+    conf.set("dt.operator.jdbcOutput.prop.tablename", TABLE_NAME);
+  }
+  
+  public int getNumOfEventsInStore()
+  {
+    Connection con;
+    try {
+      con = DriverManager.getConnection(DB_URL,USER_NAME,PSW);
+      Statement stmt = con.createStatement();
+      
+      String countQuery = "SELECT count(*) from " + TABLE_NAME;
+      ResultSet resultSet = stmt.executeQuery(countQuery);
+      resultSet.next();
+      return resultSet.getInt(1);
+    } catch (SQLException e) {
+      throw new RuntimeException("fetching count", e);
+    }
+  }
+  
+  @Test
+  public void StreamingWordExtractTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    setConfig(conf);
+    StreamingWordExtract app = new StreamingWordExtract();
+    lma.prepareDAG(app, conf);
+    LocalMode.Controller lc = lma.getController();
+    
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return getNumOfEventsInStore() == 36;
+      }
+    });
+    
+    lc.run(10000);
+  
+    Assert.assertEquals(app.getWordCount(), getNumOfEventsInStore());
+    Assert.assertEquals(app.getEntriesMapped(), getNumOfEventsInStore());
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessionsTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessionsTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessionsTest.java
new file mode 100644
index 0000000..f8ec086
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessionsTest.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.apex.malhar.stream.sample.complete;
+
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Testing the {@link TopWikipediaSessions} Application.
+ */
+public class TopWikipediaSessionsTest
+{
+  @Test
+  public void TopWikipediaSessionsTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    lma.prepareDAG(new TopWikipediaSessions(), conf);
+    LocalMode.Controller lc = lma.getController();
+    
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return TopWikipediaSessions.SessionGen.getTupleCount() >= 250;
+      }
+    });
+    
+    lc.run(30000);
+    
+    for (int i = 0; i < TopWikipediaSessions.Collector.getResult().size(); i++) {
+      Assert.assertTrue(isInOrder(TopWikipediaSessions.Collector.getResult().get(i)));
+    }
+  }
+  
+  public boolean isInOrder(List<TopWikipediaSessions.TempWrapper> input)
+  {
+    if (input.size() == 0 || input.size() == 1) {
+      return true;
+    }
+    for (int i = 0; i < input.size() - 2; i++) {
+      if (input.get(i).getValue().getValue() < input.get(i + 1).getValue().getValue()) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutesTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutesTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutesTest.java
new file mode 100644
index 0000000..e363ca7
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutesTest.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.apex.malhar.stream.sample.complete;
+
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.lib.util.KeyValPair;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Testing the {@link TrafficRoutes} Application.
+ */
+public class TrafficRoutesTest
+{
+
+  @Test
+  public void TrafficRoutesTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    lma.prepareDAG(new TrafficRoutes(), conf);
+    LocalMode.Controller lc = lma.getController();
+    
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return TrafficRoutes.InfoGen.getTupleCount() >= 100;
+      }
+    });
+    
+    lc.run(60000);
+    
+    Assert.assertTrue(!TrafficRoutes.Collector.getResult().isEmpty());
+    for (Map.Entry<KeyValPair<Long, String>, KeyValPair<Double, Boolean>> entry : TrafficRoutes.Collector.getResult().entrySet()) {
+      Assert.assertTrue(entry.getValue().getKey() <= 75);
+      Assert.assertTrue(entry.getValue().getKey() >= 55);
+      Assert.assertTrue(entry.getKey().getValue().equals("SDRoute1") || entry.getKey().getValue().equals("SDRoute2"));
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoCompleteTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoCompleteTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoCompleteTest.java
new file mode 100644
index 0000000..9ba2f25
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoCompleteTest.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.apex.malhar.stream.sample.complete;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.LocalMode;
+
+/**
+ * Testing the TwitterAutoComplete Application. In order to run this test, you need to create an app
+ * at https://apps.twitter.com, then generate your consumer and access keys and tokens, and set the following properties
+ * for the application before running it:
+ * Your application consumer key,
+ * Your application consumer secret,
+ * Your twitter access token, and
+ * Your twitter access token secret.
+ *
+ * This test is mainly for local demonstration purpose. Default time to run the application is 1 minute, please
+ * set the time you need to run the application before you run.
+ */
+public class TwitterAutoCompleteTest
+{
+  private static final Logger logger = LoggerFactory.getLogger(org.apache.apex.malhar.stream.sample.complete.AutoCompleteTest.class);
+
+  @Test
+  @Ignore
+  public void TwitterAutoCompleteTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    //uncomment the following lines and change YOUR_XXX to the corresponding information needed.
+    //conf.set("dt.application.TwitterAutoComplete.operator.tweetSampler.consumerKey", "YOUR_CONSUMERKEY");
+    //conf.set("dt.application.TwitterAutoComplete.operator.tweetSampler.consumerSecret", "YOUR_CONSUERSECRET");
+    //conf.set("dt.application.TwitterAutoComplete.operator.tweetSampler.accessToken", "YOUR_ACCESSTOKEN");
+    //conf.set("dt.application.TwitterAutoComplete.operator.tweetSampler.accessTokenSecret", "YOUR_TOKENSECRET");
+    lma.prepareDAG(new TwitterAutoComplete(), conf);
+    LocalMode.Controller lc = lma.getController();
+    long start = System.currentTimeMillis();
+    lc.run(60000); // Set your desired time to run the application here.
+    long end = System.currentTimeMillis();
+    long time = end - start;
+    logger.info("Test used " + time + " ms");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamplesTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamplesTest.java
new file mode 100644
index 0000000..5858013
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamplesTest.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.apex.malhar.stream.sample.cookbook;
+
+import java.util.concurrent.Callable;
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Test for {@link CombinePerKeyExamples}.
+ */
+public class CombinePerKeyExamplesTest
+{
+  @Test
+  public void CombinePerKeyExamplesTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+  
+    CombinePerKeyExamples app = new CombinePerKeyExamples();
+      
+    lma.prepareDAG(app, conf);
+    
+    LocalMode.Controller lc = lma.getController();
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return CombinePerKeyExamples.SampleInput.getI() >= 1;
+      }
+    });
+    lc.run(100000);
+  
+    Assert.assertTrue(CombinePerKeyExamples.Collector.result.get(CombinePerKeyExamples.Collector.result.size() - 1).getCorpus().contains("1, 2, 3, 4, 5, 6, 7, 8"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExampleTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExampleTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExampleTest.java
new file mode 100644
index 0000000..ed4ddb4
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/DeDupExampleTest.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.cookbook;
+
+import java.util.concurrent.Callable;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
+import com.datatorrent.api.LocalMode;
+
+import com.datatorrent.stram.StramLocalCluster;
+
+
+/**
+ * Test for {@link DeDupExample}.
+ */
+public class DeDupExampleTest
+{
+  @Test
+  public void DeDupExampleTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    
+    DeDupExample app = new DeDupExample();
+    lma.prepareDAG(app, conf);
+    LocalMode.Controller lc = lma.getController();
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return DeDupExample.Collector.isDone();
+      }
+    });
+    lc.run(50000);
+  
+    Assert.assertEquals(9, DeDupExample.Collector.getResult().getValue().size());
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamplesTest.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamplesTest.java b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamplesTest.java
new file mode 100644
index 0000000..51981de
--- /dev/null
+++ b/demos/highlevelapi/src/test/java/org/apache/apex/malhar/stream/sample/cookbook/MaxPerKeyExamplesTest.java
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.cookbook;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Throwables;
+
+import com.datatorrent.api.LocalMode;
+import com.datatorrent.lib.db.jdbc.JdbcTransactionalStore;
+import com.datatorrent.stram.StramLocalCluster;
+
+/**
+ * Test for MaxPerKeyExamples Application.
+ */
+public class MaxPerKeyExamplesTest
+{
+  
+  private static final String INPUT_TUPLE_CLASS = "org.apache.apex.malhar.stream.sample.cookbook.InputPojo";
+  private static final String OUTPUT_TUPLE_CLASS = "org.apache.apex.malhar.stream.sample.cookbook.OutputPojo";
+  private static final String DB_DRIVER = "org.h2.Driver";
+  private static final String DB_URL = "jdbc:h2:~/test";
+  private static final String INPUT_TABLE = "InputTable";
+  private static final String OUTPUT_TABLE = "OutputTable";
+  private static final String USER_NAME = "root";
+  private static final String PSW = "password";
+  private static final String QUERY = "SELECT * FROM " + INPUT_TABLE + ";";
+  
+  private static final double[] MEANTEMPS = {85.3, 75.4};
+  
+  @BeforeClass
+  public static void setup()
+  {
+    try {
+      Class.forName(DB_DRIVER).newInstance();
+      
+      Connection con = DriverManager.getConnection(DB_URL,USER_NAME,PSW);
+      Statement stmt = con.createStatement();
+      
+      String createMetaTable = "CREATE TABLE IF NOT EXISTS " + JdbcTransactionalStore.DEFAULT_META_TABLE + " ( "
+          + JdbcTransactionalStore.DEFAULT_APP_ID_COL + " VARCHAR(100) NOT NULL, "
+          + JdbcTransactionalStore.DEFAULT_OPERATOR_ID_COL + " INT NOT NULL, "
+          + JdbcTransactionalStore.DEFAULT_WINDOW_COL + " BIGINT NOT NULL, "
+          + "UNIQUE (" + JdbcTransactionalStore.DEFAULT_APP_ID_COL + ", "
+          + JdbcTransactionalStore.DEFAULT_OPERATOR_ID_COL + ", " + JdbcTransactionalStore.DEFAULT_WINDOW_COL + ") "
+          + ")";
+      stmt.executeUpdate(createMetaTable);
+      
+      String createInputTable = "CREATE TABLE IF NOT EXISTS " + INPUT_TABLE
+          + "(MONTH INT(2) not NULL, DAY INT(2), YEAR INT(4), MEANTEMP DOUBLE(10) )";
+      stmt.executeUpdate(createInputTable);
+  
+      String createOutputTable = "CREATE TABLE IF NOT EXISTS " + OUTPUT_TABLE
+          + "(MONTH INT(2) not NULL, MEANTEMP DOUBLE(10) )";
+      stmt.executeUpdate(createOutputTable);
+      
+      String cleanTable = "truncate table " + INPUT_TABLE;
+      stmt.executeUpdate(cleanTable);
+  
+      stmt = con.createStatement();
+  
+      String sql = "INSERT INTO " + INPUT_TABLE + " VALUES (6, 21, 2014, 85.3)";
+      stmt.executeUpdate(sql);
+      sql = "INSERT INTO " + INPUT_TABLE + " VALUES (7, 20, 2014, 75.4)";
+      stmt.executeUpdate(sql);
+      sql = "INSERT INTO " + INPUT_TABLE + " VALUES (6, 18, 2014, 45.3)";
+      stmt.executeUpdate(sql);
+      
+    } catch (Throwable e) {
+      throw Throwables.propagate(e);
+    }
+  }
+  
+  @AfterClass
+  public static void cleanup()
+  {
+    try {
+      Class.forName(DB_DRIVER).newInstance();
+  
+      Connection con = DriverManager.getConnection(DB_URL, USER_NAME, PSW);
+      Statement stmt = con.createStatement();
+  
+      String dropInputTable = "DROP TABLE " + INPUT_TABLE;
+      stmt.executeUpdate(dropInputTable);
+  
+      String dropOutputTable = "DROP TABLE " + OUTPUT_TABLE;
+      stmt.executeUpdate(dropOutputTable);
+      
+    } catch (Throwable e) {
+      throw Throwables.propagate(e);
+    }
+    
+  }
+  
+  public void setConfig(Configuration conf)
+  {
+    conf.set("dt.operator.jdbcInput.prop.store.userName", USER_NAME);
+    conf.set("dt.operator.jdbcInput.prop.store.password", PSW);
+    conf.set("dt.operator.jdbcInput.prop.store.databaseDriver", DB_DRIVER);
+    conf.set("dt.operator.jdbcInput.prop.batchSize", "5");
+    conf.set("dt.operator.jdbcInput.port.outputPort.attr.TUPLE_CLASS", INPUT_TUPLE_CLASS);
+    conf.set("dt.operator.jdbcInput.prop.store.databaseUrl", DB_URL);
+    conf.set("dt.operator.jdbcInput.prop.tableName", INPUT_TABLE);
+    conf.set("dt.operator.jdbcInput.prop.query", QUERY);
+  
+    conf.set("dt.operator.jdbcOutput.prop.store.userName", USER_NAME);
+    conf.set("dt.operator.jdbcOutput.prop.store.password", PSW);
+    conf.set("dt.operator.jdbcOutput.prop.store.databaseDriver", DB_DRIVER);
+    conf.set("dt.operator.jdbcOutput.prop.batchSize", "5");
+    conf.set("dt.operator.jdbcOutput.port.input.attr.TUPLE_CLASS", OUTPUT_TUPLE_CLASS);
+    conf.set("dt.operator.jdbcOutput.prop.store.databaseUrl", DB_URL);
+    conf.set("dt.operator.jdbcOutput.prop.tablename", OUTPUT_TABLE);
+  }
+  
+  public int getNumEntries()
+  {
+    Connection con;
+    try {
+      con = DriverManager.getConnection(DB_URL,USER_NAME,PSW);
+      Statement stmt = con.createStatement();
+    
+      String countQuery = "SELECT count(DISTINCT (MONTH, MEANTEMP)) from " + OUTPUT_TABLE;
+      ResultSet resultSet = stmt.executeQuery(countQuery);
+      resultSet.next();
+      return resultSet.getInt(1);
+    } catch (SQLException e) {
+      throw new RuntimeException("fetching count", e);
+    }
+  }
+  
+  public Map<Integer, Double> getMaxMeanTemp()
+  {
+    Map<Integer, Double> result = new HashMap<>();
+    Connection con;
+    try {
+      con = DriverManager.getConnection(DB_URL,USER_NAME,PSW);
+      Statement stmt = con.createStatement();
+    
+      String countQuery = "SELECT DISTINCT * from " + OUTPUT_TABLE;
+      ResultSet resultSet = stmt.executeQuery(countQuery);
+      while (resultSet.next()) {
+        result.put(resultSet.getInt("MONTH"), resultSet.getDouble("MEANTEMP"));
+        
+      }
+      return result;
+    } catch (SQLException e) {
+      throw new RuntimeException("fetching count", e);
+    }
+  }
+  
+  @Test
+  public void MaxPerKeyExampleTest() throws Exception
+  {
+    LocalMode lma = LocalMode.newInstance();
+    Configuration conf = new Configuration(false);
+    setConfig(conf);
+    
+    MaxPerKeyExamples app = new MaxPerKeyExamples();
+  
+    lma.prepareDAG(app, conf);
+  
+    LocalMode.Controller lc = lma.getController();
+    ((StramLocalCluster)lc).setExitCondition(new Callable<Boolean>()
+    {
+      @Override
+      public Boolean call() throws Exception
+      {
+        return getNumEntries() == 2;
+      }
+    });
+    
+    lc.run(5000);
+    
+    double[] result = new double[2];
+    result[0] = getMaxMeanTemp().get(6);
+    result[1] = getMaxMeanTemp().get(7);
+    Assert.assertArrayEquals(MEANTEMPS, result, 0.0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/resources/data/word.txt
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/resources/data/word.txt b/demos/highlevelapi/src/test/resources/data/word.txt
new file mode 100644
index 0000000..7e28409
--- /dev/null
+++ b/demos/highlevelapi/src/test/resources/data/word.txt
@@ -0,0 +1,2 @@
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error


[5/6] apex-malhar git commit: APEXMALHAR-2142 #comment Implement WindowedStream interface

Posted by th...@apache.org.
http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/DagMeta.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/DagMeta.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/DagMeta.java
index 791ce3e..eae8b15 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/DagMeta.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/DagMeta.java
@@ -27,8 +27,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.apex.malhar.stream.api.Option;
 import org.apache.commons.lang3.tuple.MutablePair;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.classification.InterfaceStability;
 
 import com.datatorrent.api.Attribute;
 import com.datatorrent.api.DAG;
@@ -36,14 +41,15 @@ import com.datatorrent.api.Operator;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
 
 /**
- * Graph data structure for DAG
- * With this data structure, the framework can do lazy load and optimization
+ * Logical graph data structure for DAG <br>
+ *
+ * With the build method({@link #buildDAG()}, {@link #buildDAG(DAG)}) to convert it to Apex DAG
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public class DagMeta
 {
-
   private List<NodeMeta> heads = new LinkedList<>();
 
   List<Pair<Attribute, Object>> dagAttributes = new LinkedList<>();
@@ -51,10 +57,10 @@ public class DagMeta
   public static class NodeMeta
   {
 
-    private String nodeName;
-
     private Operator operator;
 
+    private Option[] options;
+
     List<Pair<Attribute, Object>> operatorAttributes = new LinkedList<>();
 
     private Map<Operator.OutputPort, Pair<List<Operator.InputPort>, DAG.Locality>> nodeStreams = new HashMap<>();
@@ -79,11 +85,6 @@ public class DagMeta
       return children;
     }
 
-    public String getNodeName()
-    {
-      return nodeName;
-    }
-
     public Operator getOperator()
     {
       return operator;
@@ -94,13 +95,13 @@ public class DagMeta
       return nodeStreams;
     }
 
-    public NodeMeta(Operator operator, String nodeName)
+    public NodeMeta(Operator operator, Option... options)
     {
 
-      this.nodeName = nodeName;
-
       this.operator = operator;
 
+      this.options = options;
+
       for (Field field : this.operator.getClass().getFields()) {
         int modifier = field.getModifiers();
         if (Modifier.isPublic(modifier) && Modifier.isTransient(modifier) &&
@@ -122,6 +123,15 @@ public class DagMeta
       }
     }
 
+    public String getOperatorName()
+    {
+      for (Option opt : options) {
+        if (opt instanceof Option.OpName) {
+          return ((Option.OpName)opt).getName();
+        }
+      }
+      return operator.toString();
+    }
   }
 
   public DagMeta()
@@ -141,11 +151,15 @@ public class DagMeta
     for (NodeMeta nm : heads) {
       visitNode(nm, dag);
     }
+    logger.debug("Finish building the dag:\n {}", dag.toString());
   }
 
   private void visitNode(NodeMeta nm, DAG dag)
   {
-    dag.addOperator(nm.nodeName, nm.operator);
+    String opName = nm.getOperatorName();
+    logger.debug("Building DAG: add operator {}: {}", opName, nm.operator);
+    dag.addOperator(opName, nm.operator);
+
     for (NodeMeta child : nm.children) {
       visitNode(child, dag);
     }
@@ -154,15 +168,18 @@ public class DagMeta
       if (entry.getKey() == null || entry.getValue().getKey() == null || 0 == entry.getValue().getKey().size()) {
         continue;
       }
+      logger.debug("Building DAG: add stream {} from {} to {}", entry.getKey().toString(), entry.getKey(), entry.getValue().getLeft().toArray(new Operator.InputPort[]{}));
       DAG.StreamMeta streamMeta = dag.addStream(entry.getKey().toString(), entry.getKey(),
           entry.getValue().getLeft().toArray(new Operator.InputPort[]{}));
       // set locality
       if (entry.getValue().getRight() != null) {
+        logger.debug("Building DAG: set locality of the stream {} to {}", entry.getKey().toString(), entry.getValue().getRight());
         streamMeta.setLocality(entry.getValue().getRight());
       }
       //set attributes for output port
       if (nm.outputPortAttributes.containsKey(entry.getKey())) {
         for (Pair<Attribute, Object> attr : nm.outputPortAttributes.get(entry.getKey())) {
+          logger.debug("Building DAG: set port attribute {} to {} for port {}", attr.getLeft(), attr.getValue(), entry.getKey());
           dag.setOutputPortAttribute(entry.getKey(), attr.getLeft(), attr.getValue());
         }
       }
@@ -173,6 +190,7 @@ public class DagMeta
       //set input port attributes
       if (nm.inputPortAttributes.containsKey(input)) {
         for (Pair<Attribute, Object> attr : nm.inputPortAttributes.get(input)) {
+          logger.debug("Building DAG: set port attribute {} to {} for port {}", attr.getLeft(), attr.getValue(), input);
           dag.setInputPortAttribute(input, attr.getLeft(), attr.getValue());
         }
       }
@@ -180,15 +198,16 @@ public class DagMeta
 
     // set operator attributes
     for (Pair<Attribute, Object> attr : nm.operatorAttributes) {
+      logger.debug("Building DAG: set operator attribute {} to {} for operator {}", attr.getLeft(), attr.getValue(), nm.operator);
       dag.setAttribute(nm.operator, attr.getLeft(), attr.getValue());
     }
 
   }
 
-  public NodeMeta addNode(String nodeName, Operator operator, NodeMeta parent, Operator.OutputPort parentOutput, Operator.InputPort inputPort)
+  public NodeMeta addNode(Operator operator, NodeMeta parent, Operator.OutputPort parentOutput, Operator.InputPort inputPort, Option... options)
   {
 
-    NodeMeta newNode = new NodeMeta(operator, nodeName);
+    NodeMeta newNode = new NodeMeta(operator, options);
     if (parent == null) {
       heads.add(newNode);
     } else {
@@ -199,4 +218,6 @@ public class DagMeta
     return newNode;
   }
 
+  private static final Logger logger = LoggerFactory.getLogger(DagMeta.class);
+
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/IDGenerator.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/IDGenerator.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/IDGenerator.java
index 982980c..b5bc286 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/IDGenerator.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/IDGenerator.java
@@ -20,6 +20,8 @@ package org.apache.apex.malhar.stream.api.impl;
 
 import java.util.UUID;
 
+import org.apache.hadoop.classification.InterfaceStability;
+
 import com.datatorrent.api.Operator;
 
 import static java.lang.System.currentTimeMillis;
@@ -29,6 +31,7 @@ import static java.lang.System.currentTimeMillis;
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public class IDGenerator
 {
 

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java
index 7af6ece..d6201ad 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/StreamFactory.java
@@ -18,39 +18,59 @@
  */
 package org.apache.apex.malhar.stream.api.impl;
 
+import org.apache.apex.malhar.kafka.KafkaSinglePortInputOperator;
+import org.apache.apex.malhar.kafka.PartitionStrategy;
 import org.apache.apex.malhar.lib.fs.LineByLineFileInputOperator;
 import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.Option;
+import org.apache.hadoop.classification.InterfaceStability;
 
 import com.datatorrent.api.InputOperator;
 import com.datatorrent.api.Operator;
 import com.datatorrent.contrib.kafka.KafkaSinglePortStringInputOperator;
 
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
 /**
- * A Factory class to build from different kind of input source
+ * A Factory class to build stream from different input sources
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public class StreamFactory
 {
-  public static ApexStream<String> fromFolder(String inputOperatorName, String folderName)
+  /**
+   * Create a stream of string tuples from reading files in hdfs folder line by line
+   * @param folderName
+   * @param opts
+   * @return
+   */
+  public static ApexStream<String> fromFolder(String folderName, Option... opts)
   {
     LineByLineFileInputOperator fileLineInputOperator = new LineByLineFileInputOperator();
     fileLineInputOperator.setDirectory(folderName);
     ApexStreamImpl<String> newStream = new ApexStreamImpl<>();
-    return newStream.addOperator(inputOperatorName, fileLineInputOperator, null, fileLineInputOperator.output);
+    return newStream.addOperator(fileLineInputOperator, null, fileLineInputOperator.output, opts);
   }
 
   public static ApexStream<String> fromFolder(String folderName)
   {
-    return fromFolder("FolderScanner", folderName);
+    return fromFolder(folderName, name("FolderScanner"));
   }
 
   public static ApexStream<String> fromKafka08(String zookeepers, String topic)
   {
-    return fromKafka08("Kafka08Input", zookeepers, topic);
+    return fromKafka08(zookeepers, topic, name("Kafka08Input"));
   }
 
-  public static ApexStream<String> fromKafka08(String inputName, String zookeepers, String topic)
+  /**
+   * Create a stream of string reading input from kafka 0.8
+   * @param zookeepers
+   * @param topic
+   * @param opts
+   * @return
+   */
+  public static ApexStream<String> fromKafka08(String zookeepers, String topic, Option... opts)
   {
     KafkaSinglePortStringInputOperator kafkaSinglePortStringInputOperator = new KafkaSinglePortStringInputOperator();
     kafkaSinglePortStringInputOperator.getConsumer().setTopic(topic);
@@ -59,25 +79,49 @@ public class StreamFactory
     return newStream.addOperator(kafkaSinglePortStringInputOperator, null, kafkaSinglePortStringInputOperator.outputPort);
   }
 
-  public static <T> ApexStream<T> fromInput(String inputOperatorName, InputOperator operator, Operator.OutputPort<T> outputPort)
+  /**
+   * Create a stream with any input operator
+   * @param operator
+   * @param outputPort
+   * @param opts
+   * @param <T>
+   * @return
+   */
+  public static <T> ApexStream<T> fromInput(InputOperator operator, Operator.OutputPort<T> outputPort, Option... opts)
   {
     ApexStreamImpl<T> newStream = new ApexStreamImpl<>();
-    return newStream.addOperator(inputOperatorName, operator, null, outputPort);
+    return newStream.addOperator(operator, null, outputPort, opts);
   }
 
-  public static <T> ApexStream<T> fromInput(InputOperator operator, Operator.OutputPort<T> outputPort)
+  /**
+   * Create stream of byte array messages from kafka 0.9
+   * @param brokers
+   * @param topic
+   * @param opts
+   * @return
+   */
+  public static ApexStream<byte[]> fromKafka09(String brokers, String topic, Option... opts)
   {
-    return fromInput(operator.toString(), operator, outputPort);
+    KafkaSinglePortInputOperator kafkaInput = new KafkaSinglePortInputOperator();
+    kafkaInput.setClusters(brokers);
+    kafkaInput.setTopics(topic);
+    ApexStreamImpl<String> newStream = new ApexStreamImpl<>();
+    return newStream.addOperator(kafkaInput, null, kafkaInput.outputPort, opts);
   }
 
-  public static ApexStream<String> fromKafka09(String name, String brokers, String topic)
+  /**
+   * Create stream of byte array messages from kafka 0.9 with more partition options
+   */
+  public static ApexStream<byte[]> fromKafka09(String brokers, String topic, PartitionStrategy strategy, int partitionNumber, Option... opts)
   {
-    throw new UnsupportedOperationException();
+    KafkaSinglePortInputOperator kafkaInput = new KafkaSinglePortInputOperator();
+    kafkaInput.setClusters(brokers);
+    kafkaInput.setTopics(topic);
+    kafkaInput.setStrategy(strategy.name());
+    kafkaInput.setInitialPartitionCount(partitionNumber);
+    ApexStreamImpl<String> newStream = new ApexStreamImpl<>();
+    return newStream.addOperator(kafkaInput, null, kafkaInput.outputPort, opts);
   }
 
-  public static ApexStream<String> fromKafka09(String brokers, String topic)
-  {
-    return fromKafka09("KafkaInput", brokers, topic);
-  }
 
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/TupleWrapperOperator.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/TupleWrapperOperator.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/TupleWrapperOperator.java
new file mode 100644
index 0000000..1bc500d
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/TupleWrapperOperator.java
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api.impl;
+
+import javax.validation.constraints.NotNull;
+
+import org.apache.apex.malhar.lib.window.Tuple;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.Operator;
+import com.datatorrent.api.Sink;
+import com.datatorrent.api.annotation.InputPortFieldAnnotation;
+
+/**
+ * A wrapper operator that intercept the tuples and convert them between {@link Tuple}
+ */
+public class TupleWrapperOperator implements InputOperator, Operator.CheckpointNotificationListener
+{
+
+  public static class OutputPortWrapper extends DefaultOutputPort implements Sink
+  {
+
+    @Override
+    public void put(Object o)
+    {
+      emit(o);
+    }
+
+    @Override
+    public int getCount(boolean b)
+    {
+      // No Accumulation
+      return 0;
+    }
+  }
+
+  public static class InputPortWrapper extends DefaultInputPort<Tuple>
+  {
+
+    @NotNull
+    private DefaultInputPort input;
+
+    public void setInput(DefaultInputPort input)
+    {
+      this.input = input;
+    }
+
+    @Override
+    public void process(Tuple o)
+    {
+      input.process(o.getValue());
+    }
+
+    @Override
+    public Sink getSink()
+    {
+      return input.getSink();
+    }
+
+    @Override
+    public void setConnected(boolean connected)
+    {
+      input.setConnected(connected);
+    }
+
+    @Override
+    public void setup(Context.PortContext context)
+    {
+      input.setup(context);
+    }
+
+    @Override
+    public void teardown()
+    {
+      input.teardown();
+    }
+  }
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient OutputPortWrapper output1 = new OutputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient OutputPortWrapper output2 = new OutputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient OutputPortWrapper output3 = new OutputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient OutputPortWrapper output4 = new OutputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient OutputPortWrapper output5 = new OutputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient InputPortWrapper input1 = new InputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient InputPortWrapper input2 = new InputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient InputPortWrapper input3 = new InputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient InputPortWrapper input4 = new InputPortWrapper();
+
+  @InputPortFieldAnnotation(optional = true)
+  public final transient InputPortWrapper input5 = new InputPortWrapper();
+
+  //delegate to
+  @NotNull
+  private Operator operator;
+
+  public void setOperator(Operator operator)
+  {
+    this.operator = operator;
+  }
+
+  @Override
+  public void beginWindow(long l)
+  {
+    operator.beginWindow(l);
+  }
+
+  @Override
+  public void endWindow()
+  {
+    operator.endWindow();
+  }
+
+  @Override
+  public void setup(Context.OperatorContext context)
+  {
+    operator.endWindow();
+  }
+
+  @Override
+  public void teardown()
+  {
+    operator.teardown();
+  }
+
+  @Override
+  public void beforeCheckpoint(long l)
+  {
+    if (operator instanceof CheckpointNotificationListener) {
+      ((CheckpointNotificationListener)operator).beforeCheckpoint(l);
+    }
+  }
+
+  @Override
+  public void checkpointed(long l)
+  {
+    if (operator instanceof CheckpointNotificationListener) {
+      ((CheckpointNotificationListener)operator).checkpointed(l);
+    }
+  }
+
+  @Override
+  public void committed(long l)
+  {
+    if (operator instanceof CheckpointNotificationListener) {
+      ((CheckpointNotificationListener)operator).committed(l);
+    }
+  }
+
+  @Override
+  public void emitTuples()
+  {
+    if (operator instanceof InputOperator) {
+      ((InputOperator)operator).emitTuples();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java
new file mode 100644
index 0000000..68f1b9e
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.commons.lang3.mutable.MutableLong;
+
+/**
+ * Count Accumulation
+ */
+public class Count implements Accumulation<Long, MutableLong, Long>
+{
+
+  @Override
+  public MutableLong defaultAccumulatedValue()
+  {
+    return new MutableLong(0);
+  }
+
+  @Override
+  public MutableLong accumulate(MutableLong accumulatedValue, Long input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+
+  @Override
+  public MutableLong merge(MutableLong accumulatedValue1, MutableLong accumulatedValue2)
+  {
+    accumulatedValue1.add(accumulatedValue2);
+    return accumulatedValue1;
+  }
+
+  @Override
+  public Long getOutput(MutableLong accumulatedValue)
+  {
+    return accumulatedValue.getValue();
+  }
+
+  @Override
+  public Long getRetraction(Long value)
+  {
+    return -value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java
new file mode 100644
index 0000000..3ab6892
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * Fold Accumulation Adaptor class
+ */
+public abstract class FoldFn<INPUT, OUTPUT> implements Accumulation<INPUT, OUTPUT, OUTPUT>
+{
+
+  public FoldFn()
+  {
+  }
+
+  public FoldFn(OUTPUT initialVal)
+  {
+    this.initialVal = initialVal;
+  }
+
+  private OUTPUT initialVal;
+
+  @Override
+  public OUTPUT defaultAccumulatedValue()
+  {
+    return initialVal;
+  }
+
+  @Override
+  public OUTPUT accumulate(OUTPUT accumulatedValue, INPUT input)
+  {
+    return fold(accumulatedValue, input);
+  }
+
+  @Override
+  public OUTPUT getOutput(OUTPUT accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+
+  @Override
+  public OUTPUT getRetraction(OUTPUT value)
+  {
+    return null;
+  }
+
+  abstract OUTPUT fold(OUTPUT result, INPUT input);
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java
new file mode 100644
index 0000000..b4507bc
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * An easy to use reduce Accumulation
+ * @param <INPUT>
+ */
+public abstract class ReduceFn<INPUT> implements Accumulation<INPUT, INPUT, INPUT>
+{
+  @Override
+  public INPUT defaultAccumulatedValue()
+  {
+    return null;
+  }
+
+  @Override
+  public INPUT accumulate(INPUT accumulatedValue, INPUT input)
+  {
+    if (accumulatedValue == null) {
+      return input;
+    }
+    return reduce(accumulatedValue, input);
+  }
+
+  @Override
+  public INPUT merge(INPUT accumulatedValue1, INPUT accumulatedValue2)
+  {
+    return reduce(accumulatedValue1, accumulatedValue2);
+  }
+
+  @Override
+  public INPUT getOutput(INPUT accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+
+  @Override
+  public INPUT getRetraction(INPUT value)
+  {
+    return null;
+  }
+
+  public abstract INPUT reduce(INPUT input1, INPUT input2);
+
+
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java
new file mode 100644
index 0000000..77a08a6
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/TopN.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api.impl.accumulation;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * TopN accumulation
+ */
+public class TopN<T> implements Accumulation<T, List<T>, List<T>>
+{
+
+  int n;
+
+  Comparator<T> comparator;
+
+  public void setN(int n)
+  {
+    this.n = n;
+  }
+
+  public void setComparator(Comparator<T> comparator)
+  {
+    this.comparator = comparator;
+  }
+
+  @Override
+  public List<T> defaultAccumulatedValue()
+  {
+    return new LinkedList<>();
+  }
+
+  @Override
+  public List<T> accumulate(List<T> accumulatedValue, T input)
+  {
+    int k = 0;
+    for (T inMemory : accumulatedValue) {
+      if (comparator != null) {
+        if (comparator.compare(inMemory, input) < 0) {
+          break;
+        }
+      } else if (input instanceof Comparable) {
+        if (((Comparable<T>)input).compareTo(inMemory) > 0) {
+          break;
+        }
+      } else {
+        throw new RuntimeException("Tuple cannot be compared");
+      }
+      k++;
+    }
+    accumulatedValue.add(k, input);
+    if (accumulatedValue.size() > n) {
+      accumulatedValue.remove(accumulatedValue.get(accumulatedValue.size() - 1));
+    }
+    return accumulatedValue;
+  }
+
+  @Override
+  public List<T> merge(List<T> accumulatedValue1, List<T> accumulatedValue2)
+  {
+    accumulatedValue1.addAll(accumulatedValue2);
+    if (comparator != null) {
+      Collections.sort(accumulatedValue1, Collections.reverseOrder(comparator));
+    } else {
+      Collections.sort(accumulatedValue1, Collections.reverseOrder());
+    }
+    if (accumulatedValue1.size() > n) {
+      return accumulatedValue1.subList(0, n);
+    } else {
+      return accumulatedValue1;
+    }
+  }
+
+  @Override
+  public List<T> getOutput(List<T> accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+
+  @Override
+  public List<T> getRetraction(List<T> accumulatedValue)
+  {
+    return new LinkedList<>();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/AnnonymousClassModifier.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/AnnonymousClassModifier.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/AnnonymousClassModifier.java
index cc85f37..b0fe3c5 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/AnnonymousClassModifier.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/AnnonymousClassModifier.java
@@ -18,6 +18,8 @@
  */
 package org.apache.apex.malhar.stream.api.operator;
 
+import org.apache.hadoop.classification.InterfaceStability;
+
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.AnnotationVisitor;
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Attribute;
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.ClassVisitor;
@@ -33,6 +35,7 @@ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes;
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public class AnnonymousClassModifier extends ClassVisitor
 {
   private String className;

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/ByteArrayClassLoader.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/ByteArrayClassLoader.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/ByteArrayClassLoader.java
index 9194dc2..05a791c 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/ByteArrayClassLoader.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/ByteArrayClassLoader.java
@@ -21,9 +21,12 @@ package org.apache.apex.malhar.stream.api.operator;
 
 import java.util.Map;
 
+import org.apache.hadoop.classification.InterfaceStability;
+
 /**
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public class ByteArrayClassLoader extends ClassLoader
 {
   private final Map<String, byte[]> classes;

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/FunctionOperator.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/FunctionOperator.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/FunctionOperator.java
index 0a8ba55..1e2066c 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/FunctionOperator.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/operator/FunctionOperator.java
@@ -26,10 +26,11 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import javax.validation.constraints.NotNull;
-
+import org.apache.apex.malhar.lib.window.Tuple;
 import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.util.TupleUtil;
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.InterfaceStability;
 
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.ClassReader;
 import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.ClassWriter;
@@ -39,12 +40,15 @@ import com.datatorrent.api.Context;
 import com.datatorrent.api.DefaultInputPort;
 import com.datatorrent.api.DefaultOutputPort;
 import com.datatorrent.api.Operator;
+import com.datatorrent.api.annotation.InputPortFieldAnnotation;
+import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
 
 /**
  * Operators that wrap the functions
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public class FunctionOperator<OUT, FUNCTION extends Function> implements Operator
 {
   private byte[] annonymousFunctionClass;
@@ -57,8 +61,12 @@ public class FunctionOperator<OUT, FUNCTION extends Function> implements Operato
 
   protected boolean isAnnonymous = false;
 
+  @OutputPortFieldAnnotation(optional = true)
   public final transient DefaultOutputPort<OUT> output = new DefaultOutputPort<>();
 
+  @OutputPortFieldAnnotation(optional = true)
+  public final transient DefaultOutputPort<Tuple<OUT>> tupleOutput = new DefaultOutputPort<>();
+
   public FunctionOperator(FUNCTION f)
   {
     isAnnonymous = f.getClass().isAnonymousClass();
@@ -245,6 +253,7 @@ public class FunctionOperator<OUT, FUNCTION extends Function> implements Operato
 
     }
 
+    @InputPortFieldAnnotation(optional = true)
     public final transient DefaultInputPort<IN> input = new DefaultInputPort<IN>()
     {
       @Override
@@ -255,6 +264,21 @@ public class FunctionOperator<OUT, FUNCTION extends Function> implements Operato
       }
     };
 
+    @InputPortFieldAnnotation(optional = true)
+    public final transient DefaultInputPort<Tuple<IN>> tupleInput =  new DefaultInputPort<Tuple<IN>>()
+    {
+      @Override
+      public void process(Tuple<IN> t)
+      {
+        Function.MapFunction<IN, OUT> f = getFunction();
+        if (t instanceof Tuple.PlainTuple) {
+          TupleUtil.buildOf((Tuple.PlainTuple<IN>)t, f.f(t.getValue()));
+        } else {
+          output.emit(f.f(t.getValue()));
+        }
+      }
+    };
+
     public MapFunctionOperator(Function.MapFunction<IN, OUT> f)
     {
       super(f);
@@ -269,6 +293,8 @@ public class FunctionOperator<OUT, FUNCTION extends Function> implements Operato
 
     }
 
+
+    @InputPortFieldAnnotation(optional = true)
     public final transient DefaultInputPort<IN> input = new DefaultInputPort<IN>()
     {
       @Override
@@ -281,93 +307,65 @@ public class FunctionOperator<OUT, FUNCTION extends Function> implements Operato
       }
     };
 
-    public FlatMapFunctionOperator(Function.FlatMapFunction<IN, OUT> f)
-    {
-      super(f);
-    }
-  }
-
-  public static class FoldFunctionOperator<IN, OUT> extends FunctionOperator<OUT, Function.FoldFunction<IN, OUT>>
-  {
-
-    public FoldFunctionOperator()
-    {
-
-    }
-
-    @NotNull
-    private OUT foldVal;
-
-    public final transient DefaultInputPort<IN> input = new DefaultInputPort<IN>()
+    @InputPortFieldAnnotation(optional = true)
+    public final transient DefaultInputPort<Tuple<IN>> tupleInput =  new DefaultInputPort<Tuple<IN>>()
     {
       @Override
-      public void process(IN t)
+      public void process(Tuple<IN> t)
       {
-        Function.FoldFunction<IN, OUT> f = getFunction();
-        // fold the value
-        foldVal = f.fold(t, foldVal);
-        output.emit(foldVal);
+        Function.FlatMapFunction<IN, OUT> f = getFunction();
+        if (t instanceof Tuple.PlainTuple) {
+          for (OUT out : f.f(t.getValue())) {
+            tupleOutput.emit(TupleUtil.buildOf((Tuple.PlainTuple<IN>)t, out));
+          }
+        } else {
+          for (OUT out : f.f(t.getValue())) {
+            output.emit(out);
+          }
+        }
       }
     };
 
-    public FoldFunctionOperator(Function.FoldFunction<IN, OUT> f, OUT initialVal)
+    public FlatMapFunctionOperator(Function.FlatMapFunction<IN, OUT> f)
     {
       super(f);
-      this.foldVal = initialVal;
     }
   }
 
-  public static class ReduceFunctionOperator<IN> extends FunctionOperator<IN, Function.ReduceFunction<IN>>
+
+  public static class FilterFunctionOperator<IN> extends FunctionOperator<IN, Function.FilterFunction<IN>>
   {
 
-    public ReduceFunctionOperator()
+    public FilterFunctionOperator()
     {
 
     }
 
-    @NotNull
-    private IN reducedVal;
-
+    @InputPortFieldAnnotation(optional = true)
     public final transient DefaultInputPort<IN> input = new DefaultInputPort<IN>()
     {
       @Override
       public void process(IN t)
       {
-        Function.ReduceFunction<IN> f = getFunction();
+        Function.FilterFunction<IN> f = getFunction();
         // fold the value
-        if (reducedVal == null) {
-          reducedVal = t;
-          return;
+        if (f.f(t)) {
+          output.emit(t);
         }
-        reducedVal = f.reduce(t, reducedVal);
-        output.emit(reducedVal);
       }
     };
 
-    public ReduceFunctionOperator(Function.ReduceFunction<IN> f)
-    {
-      super(f);
-    }
-  }
-
-  public static class FilterFunctionOperator<IN> extends FunctionOperator<IN, Function.FilterFunction<IN>>
-  {
-
-    public FilterFunctionOperator()
-    {
-
-    }
-
-    public final transient DefaultInputPort<IN> input = new DefaultInputPort<IN>()
+    @InputPortFieldAnnotation(optional = true)
+    public final transient DefaultInputPort<Tuple<IN>> tupleInput =  new DefaultInputPort<Tuple<IN>>()
     {
       @Override
-      public void process(IN t)
+      public void process(Tuple<IN> t)
       {
         Function.FilterFunction<IN> f = getFunction();
-        // fold the value
-        if (f.f(t)) {
-          output.emit(t);
+        if (f.f(t.getValue())) {
+          tupleOutput.emit(t);
         }
+
       }
     };
 

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/util/KeyedTuple.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/util/KeyedTuple.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/util/KeyedTuple.java
deleted file mode 100644
index 3641189..0000000
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/util/KeyedTuple.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.apex.malhar.stream.api.util;
-
-/**
- * An interface indicate a tuple with a specific key
- * It is used internally to identify the key from the tuple
- *
- * @since 3.4.0
- */
-public interface KeyedTuple<K>
-{
-  /**
-   * Return the key of the tuple
-   * @return
-   */
-  K getKey();
-}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/util/TupleUtil.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/util/TupleUtil.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/util/TupleUtil.java
index 583615a..04f42b3 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/util/TupleUtil.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/util/TupleUtil.java
@@ -18,6 +18,11 @@
  */
 package org.apache.apex.malhar.stream.api.util;
 
+import java.util.List;
+
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.Window;
+
 /**
  * The tuple util will be used to extract fields that are used as key or value<br>
  * Or converting from data tuples to display tuples <br>
@@ -29,8 +34,22 @@ package org.apache.apex.malhar.stream.api.util;
 public class TupleUtil
 {
 
-  public static interface NONE
+  public static <T, O> Tuple.PlainTuple<O> buildOf(Tuple.PlainTuple<T> t, O newValue)
   {
 
+    if (t instanceof Tuple.WindowedTuple) {
+      Tuple.WindowedTuple<O> newT = new Tuple.WindowedTuple<>();
+      List<Window> wins = ((Tuple.WindowedTuple)t).getWindows();
+      for (Window w : wins) {
+        newT.addWindow(w);
+      }
+      newT.setValue(newValue);
+      ((Tuple.WindowedTuple)t).setTimestamp(((Tuple.WindowedTuple)t).getTimestamp());
+      return newT;
+    } else if (t instanceof Tuple.TimestampedTuple) {
+      return new Tuple.TimestampedTuple<>(((Tuple.TimestampedTuple)t).getTimestamp(), newValue);
+    } else {
+      return new Tuple.PlainTuple<>(newValue);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/FunctionOperator/FunctionOperatorTest.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/FunctionOperator/FunctionOperatorTest.java b/stream/src/test/java/org/apache/apex/malhar/stream/FunctionOperator/FunctionOperatorTest.java
index 34820b6..9d03f2a 100644
--- a/stream/src/test/java/org/apache/apex/malhar/stream/FunctionOperator/FunctionOperatorTest.java
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/FunctionOperator/FunctionOperatorTest.java
@@ -269,7 +269,7 @@ public class FunctionOperatorTest
         = new FunctionOperator.FilterFunctionOperator<Integer>(new Function.FilterFunction<Integer>()
         {
           @Override
-          public Boolean f(Integer in)
+          public boolean f(Integer in)
           {
             return in % divider == 0;
           }
@@ -309,7 +309,7 @@ public class FunctionOperatorTest
         .filter(new Function.FilterFunction<Integer>()
         {
           @Override
-          public Boolean f(Integer in)
+          public boolean f(Integer in)
           {
             return in % divider == 0;
           }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImplTest.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImplTest.java b/stream/src/test/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImplTest.java
index 71b9a82..99d5ca6 100644
--- a/stream/src/test/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImplTest.java
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImplTest.java
@@ -33,6 +33,8 @@ import com.datatorrent.api.DefaultOutputPort;
 import com.datatorrent.common.util.BaseOperator;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
 
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
 /**
  * Unit test to default implementation of ApexStream interface
  */
@@ -45,8 +47,8 @@ public class ApexStreamImplTest
     LogicalPlan dag = new LogicalPlan();
     TestOperator<String, Integer> firstOperator = new TestOperator<>();
     TestOperator<Integer, Date> secondOperator = new TestOperator<>();
-    new ApexStreamImpl<String>().addOperator("first", firstOperator, null, firstOperator.output)
-        .addOperator("second", secondOperator, secondOperator.input, null)
+    new ApexStreamImpl<String>().addOperator(firstOperator, null, firstOperator.output, name("first"))
+        .endWith(secondOperator, secondOperator.input, name("second"))
         .with(DAG.Locality.THREAD_LOCAL)
         .with(Context.OperatorContext.AUTO_RECORD, true)
         .with("prop", "TestProp").populateDag(dag);

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/sample/ApplicationWithStreamAPI.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/ApplicationWithStreamAPI.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/ApplicationWithStreamAPI.java
index 44f76b1..f65806e 100644
--- a/stream/src/test/java/org/apache/apex/malhar/stream/sample/ApplicationWithStreamAPI.java
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/ApplicationWithStreamAPI.java
@@ -20,6 +20,11 @@ package org.apache.apex.malhar.stream.sample;
 
 import java.util.Arrays;
 
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
 import org.apache.apex.malhar.stream.api.ApexStream;
 import org.apache.apex.malhar.stream.api.function.Function;
 import org.apache.apex.malhar.stream.api.impl.StreamFactory;
@@ -28,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import com.datatorrent.api.DAG;
 import com.datatorrent.api.StreamingApplication;
 import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.lib.util.KeyValPair;
 
 /**
  * An application example with stream api
@@ -40,18 +46,27 @@ public class ApplicationWithStreamAPI implements StreamingApplication
   public void populateDAG(DAG dag, Configuration configuration)
   {
     String localFolder = "./src/test/resources/data";
-    ApexStream stream = StreamFactory
+    ApexStream<String> stream = StreamFactory
         .fromFolder(localFolder)
         .flatMap(new Function.FlatMapFunction<String, String>()
         {
           @Override
           public Iterable<String> f(String input)
           {
-            return Arrays.asList(input.split(" "));
+            return Arrays.asList(input.split("[\\p{Punct}\\s]+"));
           }
         });
     stream.print();
-    stream.countByKey().print();
+    stream.window(new WindowOption.GlobalWindow(), new TriggerOption().withEarlyFiringsAtEvery(Duration
+        .millis(1000)).accumulatingFiredPanes()).countByKey(new Function.ToKeyValue<String, String, Long>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, Long>> f(String input)
+          {
+            return new Tuple.PlainTuple(new KeyValPair<>(input, 1L));
+          }
+        }).print();
     stream.populateDag(dag);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/sample/LocalTestWithoutStreamApplication.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/LocalTestWithoutStreamApplication.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/LocalTestWithoutStreamApplication.java
index d679135..f46fb14 100644
--- a/stream/src/test/java/org/apache/apex/malhar/stream/sample/LocalTestWithoutStreamApplication.java
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/LocalTestWithoutStreamApplication.java
@@ -27,9 +27,13 @@ import java.util.concurrent.Callable;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
 import org.apache.apex.malhar.stream.api.function.Function;
 import org.apache.apex.malhar.stream.api.impl.StreamFactory;
 
+import com.datatorrent.lib.util.KeyValPair;
 
 /**
  * A embedded application test without creating Streaming Application
@@ -41,25 +45,32 @@ public class LocalTestWithoutStreamApplication
   public void testNonStreamApplicationWordcount() throws Exception
   {
 
-    TupleCollector<Map<Object, Integer>> collector = new TupleCollector<>();
+    TupleCollector<Tuple.WindowedTuple<KeyValPair<String, Long>>> collector = new TupleCollector<>();
     collector.id = "testNonStreamApplicationWordcount";
-    final Map<Object, Integer> expected = new HashMap<>();
-    expected.put("error", 2);
-    expected.put("word1", 4);
-    expected.put("word2", 8);
-    expected.put("word3", 4);
-    expected.put("word4", 4);
-    expected.put("word5", 4);
-    expected.put("word7", 4);
-    expected.put("word9", 6);
+    final Map<String, Long> expected = new HashMap<>();
+    expected.put("error", 2L);
+    expected.put("word1", 4L);
+    expected.put("word2", 8L);
+    expected.put("word3", 4L);
+    expected.put("word4", 4L);
+    expected.put("word5", 4L);
+    expected.put("word7", 4L);
+    expected.put("word9", 6L);
 
     Callable<Boolean> exitCondition = new Callable<Boolean>()
     {
       @Override
       public Boolean call() throws Exception
       {
-        List<Map<Object, Integer>> data = (List<Map<Object, Integer>>)TupleCollector.results.get("testNonStreamApplicationWordcount");
-        return (data != null) && data.size() >= 1 && expected.equals(data.get(data.size() - 1));
+        if (!TupleCollector.results.containsKey("testNonStreamApplicationWordcount") || TupleCollector.results.get("testNonStreamApplicationWordcount").isEmpty()) {
+          return false;
+        }
+        Map<String, Long> data = new HashMap<>();
+        for (Tuple.TimestampedTuple<KeyValPair<String, Long>> entry :
+            (List<Tuple.TimestampedTuple<KeyValPair<String, Long>>>)TupleCollector.results.get("testNonStreamApplicationWordcount")) {
+          data.put(entry.getValue().getKey(), entry.getValue().getValue());
+        }
+        return data.size() >= 8 && expected.equals(data);
       }
     };
 
@@ -73,13 +84,26 @@ public class LocalTestWithoutStreamApplication
             return Arrays.asList(input.split(" "));
           }
         })
-        .countByKey().addOperator(collector, collector.inputPort, collector.outputPort).print().runEmbedded(false, 30000, exitCondition);
+        .window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
+        .countByKey(new Function.ToKeyValue<String, String, Long>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, Long>> f(String input)
+          {
+            return new Tuple.PlainTuple(new KeyValPair<>(input, 1L));
+          }
+        }).addOperator(collector, collector.inputPort, null).runEmbedded(false, 30000, exitCondition);
 
+    Map<String, Long> data = new HashMap<>();
 
-    List<Map<Object, Integer>> data = (List<Map<Object, Integer>>)TupleCollector.results.get("testNonStreamApplicationWordcount");
+    for (Tuple.TimestampedTuple<KeyValPair<String, Long>> entry :
+        (List<Tuple.TimestampedTuple<KeyValPair<String, Long>>>)TupleCollector.results.get("testNonStreamApplicationWordcount")) {
+      data.put(entry.getValue().getKey(), entry.getValue().getValue());
+    }
 
+    //Thread.sleep(100000);
     Assert.assertNotNull(data);
     Assert.assertTrue(data.size() > 1);
-    Assert.assertEquals(expected, data.get(data.size() - 1));
+    Assert.assertEquals(expected, data);
   }
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java
index 4958a8e..20d7aed 100644
--- a/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStream.java
@@ -18,7 +18,6 @@
  */
 package org.apache.apex.malhar.stream.sample;
 
-import org.apache.apex.malhar.stream.api.ApexStream;
 import org.apache.apex.malhar.stream.api.function.Function;
 import org.apache.apex.malhar.stream.api.impl.ApexStreamImpl;
 
@@ -30,7 +29,7 @@ import com.datatorrent.api.DAG;
 public class MyStream<T> extends ApexStreamImpl<T>
 {
 
-  public MyStream(ApexStream<T> apexStream)
+  public MyStream(ApexStreamImpl<T> apexStream)
   {
     super(apexStream);
   }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStreamTest.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStreamTest.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStreamTest.java
index b2d1e8b..5e48974 100644
--- a/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStreamTest.java
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/MyStreamTest.java
@@ -24,37 +24,51 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
+import org.joda.time.Duration;
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
 import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.ApexStreamImpl;
 import org.apache.apex.malhar.stream.api.impl.StreamFactory;
 
+import com.datatorrent.lib.util.KeyValPair;
+
 /**
  * A test class which test your own stream implementation build on default one
  */
 @SuppressWarnings("unchecked")
 public class MyStreamTest
 {
-  static Map<Object, Integer> expected = new HashMap<>();
+  static Map<String, Long> expected = new HashMap<>();
   static String testId = null;
   static Callable<Boolean> exitCondition = null;
   static {
-    expected.put("newword1", 4);
-    expected.put("newword2", 8);
-    expected.put("newword3", 4);
-    expected.put("newword4", 4);
-    expected.put("newword5", 4);
-    expected.put("newword7", 4);
-    expected.put("newword9", 6);
+    expected.put("newword1", 4L);
+    expected.put("newword2", 8L);
+    expected.put("newword3", 4L);
+    expected.put("newword4", 4L);
+    expected.put("newword5", 4L);
+    expected.put("newword7", 4L);
+    expected.put("newword9", 6L);
 
     exitCondition = new Callable<Boolean>()
     {
       @Override
       public Boolean call() throws Exception
       {
-        List<Map<Object, Integer>> data = (List<Map<Object, Integer>>)TupleCollector.results.get(testId);
-        return (data != null) && data.size() >= 1 && expected.equals(data.get(data.size() - 1));
+        if (!TupleCollector.results.containsKey(testId) || TupleCollector.results.get(testId).isEmpty()) {
+          return false;
+        }
+        Map<String, Long> dataMap = new HashMap<>();
+        List<Tuple.TimestampedTuple<KeyValPair<String, Long>>> data = (List<Tuple.TimestampedTuple<KeyValPair<String, Long>>>)TupleCollector.results.get(testId);
+        for (Tuple.TimestampedTuple<KeyValPair<String, Long>> entry : data) {
+          dataMap.put(entry.getValue().getKey(), entry.getValue().getValue());
+        }
+        return (dataMap != null) && dataMap.size() >= 1 && expected.equals(dataMap);
       }
     };
   }
@@ -65,9 +79,9 @@ public class MyStreamTest
 
     testId = "testMethodChainWordcount";
 
-    TupleCollector<Map<Object, Integer>> collector = new TupleCollector<>();
+    TupleCollector<Tuple.WindowedTuple<KeyValPair<String, Long>>> collector = new TupleCollector<>();
     collector.id = testId;
-    new MyStream<>(StreamFactory.fromFolder("./src/test/resources/data"))
+    new MyStream<>((ApexStreamImpl<String>)StreamFactory.fromFolder("./src/test/resources/data"))
         .<String, MyStream<String>>flatMap(new Function.FlatMapFunction<String, String>()
         {
           @Override
@@ -85,17 +99,28 @@ public class MyStreamTest
         }, new Function.FilterFunction<String>()
         {
           @Override
-          public Boolean f(String input)
+          public boolean f(String input)
           {
             return input.startsWith("word");
           }
-        }).countByKey()
-        .addOperator(collector, collector.inputPort, collector.outputPort).print().runEmbedded(false, 30000, exitCondition);
+        }).window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(Duration.millis(1000)))
+        .countByKey(new Function.ToKeyValue<String, String, Long>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, Long>> f(String input)
+          {
+            return new Tuple.PlainTuple(new KeyValPair<>(input, 1L));
+          }
+        }).addOperator(collector, collector.inputPort, null)
+        .runEmbedded(false, 30000, exitCondition);
 
+    Map<String, Long> dataMap = new HashMap<>();
+    for (Tuple.TimestampedTuple<KeyValPair<String, Long>> entry : (List<Tuple.TimestampedTuple<KeyValPair<String, Long>>>)TupleCollector.results.get(testId)) {
+      dataMap.put(entry.getValue().getKey(), entry.getValue().getValue());
+    }
 
-    List<Map<Object, Integer>> data = (List<Map<Object, Integer>>)TupleCollector.results.get(testId);
-    Assert.assertTrue(data.size() > 1);
-    Assert.assertEquals(expected, data.get(data.size() - 1));
+    Assert.assertTrue(dataMap.size() > 1);
+    Assert.assertEquals(expected, dataMap);
   }
 
   @Test
@@ -103,9 +128,9 @@ public class MyStreamTest
   {
     testId = "testNonMethodChainWordcount";
 
-    TupleCollector<Map<Object, Integer>> collector = new TupleCollector<>();
+    TupleCollector<Tuple.WindowedTuple<KeyValPair<String, Long>>> collector = new TupleCollector<>();
     collector.id = testId;
-    MyStream<String> mystream = new MyStream<>(StreamFactory
+    MyStream<String> mystream = new MyStream<>((ApexStreamImpl<String>)StreamFactory
         .fromFolder("./src/test/resources/data"))
         .flatMap(new Function.FlatMapFunction<String, String>()
         {
@@ -125,16 +150,28 @@ public class MyStreamTest
     }, new Function.FilterFunction<String>()
     {
       @Override
-      public Boolean f(String input)
+      public boolean f(String input)
       {
         return input.startsWith("word");
       }
-    }).countByKey().addOperator(collector, collector.inputPort, collector.outputPort).print().runEmbedded(false, 30000, exitCondition);
+    }).window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(Duration.millis(1000)))
+    .countByKey(new Function.ToKeyValue<String, String, Long>()
+    {
+      @Override
+      public Tuple<KeyValPair<String, Long>> f(String input)
+      {
+        return new Tuple.PlainTuple(new KeyValPair<>(input, 1L));
+      }
+    }).addOperator(collector, collector.inputPort, collector.outputPort).print().runEmbedded(false, 30000, exitCondition);
+
 
+    Map<String, Long> dataMap = new HashMap<>();
+    for (Tuple.TimestampedTuple<KeyValPair<String, Long>> entry : (List<Tuple.TimestampedTuple<KeyValPair<String, Long>>>)TupleCollector.results.get(testId)) {
+      dataMap.put(entry.getValue().getKey(), entry.getValue().getValue());
+    }
 
-    List<Map<Object, Integer>> data = (List<Map<Object, Integer>>)TupleCollector.results.get(testId);
-    Assert.assertTrue(data.size() > 1);
-    Assert.assertEquals(expected, data.get(data.size() - 1));
+    Assert.assertTrue(dataMap.size() > 1);
+    Assert.assertEquals(expected, dataMap);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/sample/TupleCollector.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/TupleCollector.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/TupleCollector.java
index a5c644d..94667c9 100644
--- a/stream/src/test/java/org/apache/apex/malhar/stream/sample/TupleCollector.java
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/TupleCollector.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import com.datatorrent.api.DefaultInputPort;
 import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.annotation.OutputPortFieldAnnotation;
 import com.datatorrent.common.util.BaseOperator;
 
 /**
@@ -37,6 +38,7 @@ public class TupleCollector<T> extends BaseOperator
 
   public final transient CollectorInputPort<T> inputPort = new CollectorInputPort<>(this);
 
+  @OutputPortFieldAnnotation(optional = true)
   public final transient DefaultOutputPort<T> outputPort = new DefaultOutputPort<>();
 
   public String id = "";

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/sample/WCInput.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/WCInput.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/WCInput.java
new file mode 100644
index 0000000..14ff066
--- /dev/null
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/WCInput.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import com.google.common.base.Throwables;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.common.util.BaseOperator;
+
+public class WCInput extends BaseOperator implements InputOperator
+{
+  public final transient DefaultOutputPort<String> output = new DefaultOutputPort<>();
+
+  private transient BufferedReader reader;
+
+  @Override
+  public void setup(Context.OperatorContext context)
+  {
+    initReader();
+  }
+
+  private void initReader()
+  {
+    try {
+      Path myPath = new Path("/user/siyuan/wc/wordcount");
+      FileSystem fs = FileSystem.get(new Configuration());
+      reader = new BufferedReader(new InputStreamReader(fs.open(myPath)));
+    } catch (Exception ex) {
+      throw Throwables.propagate(ex);
+    }
+  }
+
+  @Override
+  public void teardown()
+  {
+    IOUtils.closeQuietly(reader);
+  }
+
+  @Override
+  public void emitTuples()
+  {
+    try {
+      String line = reader.readLine();
+      if (line == null) {
+        reader.close();
+        initReader();
+      } else {
+        // simulate late data
+        //long timestamp = System.currentTimeMillis() - (long)(Math.random() * 30000);
+
+        this.output.emit(line);
+      }
+    } catch (IOException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  @Override
+  public void endWindow()
+  {
+    //this.controlOutput.emit(new WatermarkImpl(System.currentTimeMillis() - 15000));
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/java/org/apache/apex/malhar/stream/sample/WordCountWithStreamAPI.java
----------------------------------------------------------------------
diff --git a/stream/src/test/java/org/apache/apex/malhar/stream/sample/WordCountWithStreamAPI.java b/stream/src/test/java/org/apache/apex/malhar/stream/sample/WordCountWithStreamAPI.java
new file mode 100644
index 0000000..11dabe4
--- /dev/null
+++ b/stream/src/test/java/org/apache/apex/malhar/stream/sample/WordCountWithStreamAPI.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample;
+
+import java.util.Arrays;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.lib.util.KeyValPair;
+
+/**
+ * Word count with streaming API
+ */
+@ApplicationAnnotation(name = "WCDemo")
+public class WordCountWithStreamAPI implements StreamingApplication
+{
+
+  @Override
+  public void populateDAG(DAG dag, Configuration configuration)
+  {
+    WCInput wcInput = new WCInput();
+    ApexStream<String> stream = StreamFactory
+        .fromInput(wcInput, wcInput.output)
+        .flatMap(new Function.FlatMapFunction<String, String>()
+        {
+          @Override
+          public Iterable<String> f(String input)
+          {
+            return Arrays.asList(input.split("[\\p{Punct}\\s]+"));
+          }
+        });
+    stream.print();
+    stream.window(new WindowOption.GlobalWindow(), new TriggerOption().withEarlyFiringsAtEvery(Duration
+        .millis(1000)).accumulatingFiredPanes())
+        .countByKey(new Function.ToKeyValue<String, String, Long>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, Long>> f(String input)
+          {
+            return new Tuple.PlainTuple(new KeyValPair<>(input, 1L));
+          }
+        }).print();
+    stream.populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/test/resources/sampletweets.txt
----------------------------------------------------------------------
diff --git a/stream/src/test/resources/sampletweets.txt b/stream/src/test/resources/sampletweets.txt
new file mode 100644
index 0000000..379d424
--- /dev/null
+++ b/stream/src/test/resources/sampletweets.txt
@@ -0,0 +1,207 @@
+Tweet content
+"Apple has $233 billion in cash. It could buy all
+
+\u2014@NFL teams
+\u2014@NBA teams
+\u2014@MLB teams
+\u2014@NHL teams
+
+...and still have $80 billion left. $AAPL"
+Read $MRNJ #NEWS, $HEMP &amp; $GRCU r above .01, that's where we r goin\U0001f680 $SPY $MSFT $SBUX $SFOR $VRX $AAPL $TSLA $GOOG $FB $EURUSD $USDJPY $MLCG
+$RXSF is leadin their sector accordin 2 @EdisonMediaCen $AAPL $SPY $TSLA $FB $EURUSD $ALK $IBB $EW $AMZN $GBPUSD $GM https://t.co/LYY2mHn755
+Philstockworld Top Trade Review $AAPL $MSFT #Dividends $USO $HOV $TWTR -- https://t.co/JArXsIm7CI https://t.co/kRR9ezhm9E
+Philstockworld Top Trade Review: $AAPL $ABX $BA $CAKE $CMG $DIS $IBM $GILD $LL $UNG $SPY -- https://t.co/EX5SYjdwBC https://t.co/7FBZwVZ63v
+"Monday\u2019s Oil Mess: Rent-A-Rebel Jacks up Prices into the Holiday $USO $AAPL 
+#Earnings -- https://t.co/cGHB3WDKA8 https://t.co/JFZIBcom1n"
+Meaningless Monday Market Movement! $AAPL $SQQQ #oil #Brexit https://t.co/j4Iqg7E1HN
+"S&amp;P Futures Back over 2,050, for Now
+$SPY $AAPL $SQQQ #China #Debt #Hedging -- https://t.co/2dOc5T89S3 https://t.co/TDPVdNRNQF"
+"\U0001f4a5TURN YOUR $500 INTO $5,000+\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE\u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $AAPL $GSAT $MGT 
+https://t.co/lwAGjfmIP3"
+Trendless Tuesday - Watch Yesterday\u2019s Fake Gains Disappear $AAPL #China $FXI #Earnings -- https://t.co/GpgGqoOlFn https://t.co/FRuixv5aZF
+"\U0001f4a5TURN YOUR $500 INTO $5,000+\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE\u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $AAPL $UVXY $JDST
+https://t.co/lwAGjfmIP3"
+"Apple has $233 billion in cash. It could buy:
+
+Uber
+Tesla 
+Twitter 
+Airbnb
+Netflix
+Yahoo
+
+...and still have $18 billion left. $AAPL"
+Option Opportunity Portfolio May Review \u2013 Up 19.3% In 30 Days! $ABX $FCX $USO $AAPL $DIS - https://t.co/rp3kMsRZ3E https://t.co/TKkc15pKcR
+Waiting for the Fed \u2013 Apple Gives Us Huge Wins: $AAPL $SQQQ #GDP #Nikkei #Futures #Oil -- https://t.co/Al3pkf350V https://t.co/LktIRF4F2b
+Tempting Tuesday - S&amp;P 2,100 is Still the Line to Watch Ahead of the Fed $AAPL $QQQ -- https://t.co/t1eDfKHJnk https://t.co/BAW3RAe7SC
+Our $SQQQ Hedge is Up 314% and Our Futures Are Up $4,850, You're Welcome!  $AAPL -- https://t.co/eUQ2kCkCOY https://t.co/Yk98oyqMZl
+"TURN YOUR \U0001f4b2500 INTO \U0001f4b25,000$\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE \u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $TWTR $AAPL $LNKD
+https://t.co/euJFNQX1g4"
+"TURN YOUR \U0001f4b2500 INTO \U0001f4b25,000$\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE \u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $TALK $PPPI $AAPL https://t.co/oSn11kxftM"
+Bears today. We getting paid! $AAPL $TWTR $BWLD $NFLX https://t.co/CCi0S3skJJ
+"Apple has $233 billion in cash. It could buy all
+
+\u2014@NFL teams
+\u2014@NBA teams
+\u2014@MLB teams
+\u2014@NHL teams
+
+...and still have $80 billion left. $AAPL"
+Are you in Sync with the market? https://t.co/ZtHHCrSAf8 #stocks #finance #investing #trading $AAPL $LNKD $NFLX  $GOOGL $FB
+The Last Time These Insiders Purchased This Stock It Sky Rocketed 1000%+ https://t.co/bmNAHBoQBD $DIA $QQQ $SPY $GOOG $AAPL $BAC $TWTR $FB
+"This Hacker Made Amazon\u2019s Alexa, Google Now, and Siri Command One Another
+https://t.co/YXP3yqmf4H $AAPL $AMZN $GOOG https://t.co/NG7r6qgfRt"
+"Over the last 3 years, the top 14 automakers upped their combined R&amp;D spend by $192 million. 
+
+$AAPL upped R&amp;D spend by $5 billion. 
+
+- MS"
+Volatility can be your friend. https://t.co/aHz2r8HHD2 #stocks #trading #investing #financials #learntodaytrade $FB $AAPL $LNKD $NFLX
+"PERCENTAGE of Apple's Revenues:
+FY 2006: 
+iPod 40%
+Mac 38%
+Services 10%
+Others 12%
+
+FY 2015:
+iPhone 66%
+Mac 11%
+iPad 10%
+Others 13%
+
+$AAPL"
+Apple recovered $40 million worth of gold from recycled iPhones, iPads &amp; Macs in 2015. https://t.co/XPBWlM6cBs $AAPL https://t.co/P0LMSRw7Ot
+"Apple's iPhone sales sink for 1st time ever last quarter
+https://t.co/TAKjUwl4Yc  @DavidGoldmanCNN @cnntech  $AAPL https://t.co/OrDp4BDpsD"
+$BAC is down 5% since our article was posted on Friday https://t.co/al8AgaSsiI $DIA $QQQ $SPY $AAPL $GOOG $FB $TWTR $STUDY $NFLX $LNKD $IBM
+Ben Franklin: The First Proponent Of Dividend Growth Investing? https://t.co/dx7FE2G9AH $AAPL $ACN $AL $BEN $CSV $HON $IJR $JNJ $JWN $PEGI
+$5,000 Friday the 13th - Yesterday's Futures Trades Pay Off Nicely $USO $SPY $AAPL  -- https://t.co/3RUEjAq1bO https://t.co/2L7cdebTlT
+I DON'T SEE ANY BUBBLE RIGHT NOW , I SWEAR ! $SPX $SPY $DIA $DJI $AAPL  $VIX $TVIX $C $BAC $GM $GE $FB #STOCKMARKET https://t.co/E5954RIpC7
+Terrible $AAPL quarter, finally. On the way to becoming $NOK. Tech is mean reverting, today's leaders are almost always tomorrow's laggards.
+The iPhone 7S could look radically different from the iPhones of today https://t.co/eQxUMAZ4eM $AAPL https://t.co/HIH3QqKpIC
+"No Bull: The Evidence
+https://t.co/Md2SNpjdwd
+$SPX $MSFT $GOOGL $AAPL $NFLX $AMZN $FB $DIS $V $BAC $GS $WMT $SBUX https://t.co/1oISHNX4cJ"
+The iPhone 7S could look radically different from the iPhones of today https://t.co/KgeVSjmcGe $AAPL https://t.co/7hFtg37oJu
+There was a 3rd Apple founder, Ronald Wayne, who sold his 10% stake for $800 in 1976. Today his share would've been worth $65 Billion. $AAPL
+Twitter Stock Set to Breakout Soon https://t.co/u4V6ChhpOW $TWTR $DIA $QQQ $SPY $AAPL $GLD $GDX $NUGT $DUST $BAC $GOOG $FB $STUDY $NFLX $IBM
+Alibaba Stock Price Breaks The 50 Day Moving Average https://t.co/ABOVWI6j2G $BABA $AAPL $YHOO $COST $UWTI $CSC $MON https://t.co/VlWGDxrQXh
+I still can\u2019t shake the feeling that $AAPL is slowly taking themselves private. https://t.co/XIAMvppDWh https://t.co/kdMGCGbMaJ
+$SPX ROADMAP 2016 #STOCKMARKET $INTC $F $SPY $AAPL $AMZN $C $VIX $FB $TWTR $GOOGL $UVXY $FAZ $FEZ $MSFT $GS $BAC $AA https://t.co/owuQ9awcDw
+"Want to know why $GOOG is so impressive and why $AAPL is so fucked? Read this years founders' letter from $GOOG:
+
+https://t.co/LiBjGZwyKw"
+"GET READY. Here are the companies reporting earnings next week: https://t.co/NXptPkQX70
+
+$AAPL $FB $TWTR $CMG $GILD https://t.co/tcIoCZdOZi"
+$SPX THIS TIME IT'S DIFFERENT! $SPY $DIA $SDOW $S  $FAZ $FEZ $AAPL $MSFT $BAC $C $JPM $GS $SIRI $AMZN $F $VIX $TVIX https://t.co/pkYVgNKv3P
+$SPX ROADMAP 2016 #STOCKS $TVIX $VXX $VALE $AAPL $AKS $FCX $MSFT $AA $MU $VIX $SPX $SPY #TRADING $PCLN $SIRI $ MCD https://t.co/6UH5He38h1
+The iPhone 6S is the first iPhone ever to sell fewer models than its predecessor https://t.co/s8iQOvPQeR $AAPL https://t.co/QmQROtQ9vY
+11/ For example, buy an Echo and see your behavior change. The future is happening, and $AAPL seems, to me, asleep.
+$RLYP $SPY $KORS $WDAY $MSFT $AAPL $QLIK $TIVO $NXPI $CPXX $AVGO $ZOES $LE $TICC $SLB $FCEL $VRA $MLNX $ASNA $ICPT https://t.co/LXMpz4rFG0
+#STOCKMARKET GRAVITY LESSONS: what goes up  must come down $SPX $SPY $DIA $QQQ $TVIX $VIX $AAPL $C $FB $PCLN $BAC $F https://t.co/8HQHBEgSj5
+Should Icahn's exit or Buffett's entry affect your $AAPL judgment? The Big Name Effect. https://t.co/9Z2ok61MUh https://t.co/udAQLfdJFe
+Apple revenue drops 13 percent, ending 13 years of growth. Greater China was especially weak, down 26 percent. $AAPL https://t.co/q4ovXUenBU
+It was a $18 billion day for Apple. https://t.co/iRbGeoTmCJ $AAPL
+"Apple has $233 billion in cash. It could buy:
+
+Uber
+Tesla 
+Twitter 
+Airbnb
+Netflix
+Yahoo
+
+...and still have $18 billion left. $AAPL"
+#3 TOP 2111.05 #STOCKS #STOCKMARKET #TRADING $SPX $SPY $VIX $TVIX $AAPL $SIRI $C $BAC $JPM $AMZN $MSFT $FB $TWTR $F https://t.co/gSqmN0fVON
+Google #IO16: Android's failure to innovate hands a Apple free run at WWDC $GOOG $AAPL https://t.co/FTs9M8JD5g https://t.co/20uou1gUkW
+$SPX 2134.72..2116.48...2111.05 HOUSTON WE HAVE A PROBLEM ! #STOCKMARKET $VIX $SPY $DIA $AAPL $C $BAC $FB $VXX $MSFT https://t.co/du3QfPUM4Q
+top #earnings $FB $AAPL $AMZN $TWTR $CMG $F $GILD $LNKD $FCX $CELG $SWKS $JBLU $T $NXPI $BA  https://t.co/lObOE0uRjZ https://t.co/94F6GJc3hE
+The iPhone 6S is the first iPhone ever to sell fewer models than its predecessor https://t.co/ZVeQ9a4Yrh $AAPL https://t.co/2Ntpbxwlyo
+You do not want to miss this incredibly candid look into $AAPL w/ @tim_cook! Tune into @MadMoneyOnCNBC on @CNBC now! https://t.co/budv4qfvju
+Foxconn axes 60,000 jobs in one Chinese factory as robots take over: https://t.co/BnFdjGCmLf $AAPL https://t.co/WhRHer8jdN
+Warren Buffett's Berkshire Hathaway reports 9.8M share stake in $AAPL https://t.co/nXmvK6PV7M https://t.co/MAcMz0iTg6
+Apple is about to report its worst quarter in 13 years on Tuesday https://t.co/NJ3hwunHCx $AAPL https://t.co/YLTmnpqNjI
+Everyone who wants an iPhone has one. $AAPL is now a consumer staple stock and will trade on replacement / shareholder yield.
+Financial Armageddon\u2019 is imminent, the next major crash will happen in 2016 $VXX $VIX $TVIX  $SPX $SPY $AAPL $MSFT $BAC $C $FB $DJI $DIA $F
+"Apple is NO longer the largest US stock by market cap. Google is: https://t.co/i81Y83jQJC
+
+$GOOGL $AAPL https://t.co/cRCKRYBICS"
+Exclusive: Apple hires former Tesla VP Chris Porritt for \u2018special [car] project\u2019 https://t.co/7knsloxvJW $TSLA $AAPL https://t.co/X8cYztExoP
+$SPX on the top of downtrend Channel Be careful! #STOCKMARKET $SPY $AAPL $AMZN $TSLA $FB $QQQ $DIA $NFLX $PCLN $C $F https://t.co/UKZCyLYuBq
+UPDATE: Apple CEO Cook says in conference call that smartphone marker is 'currently not growing' $AAPL https://t.co/WeECmrdv1j
+In February Charlie Munger was asked why Berkshire owns $GM. The $AAPL stake isn't anymore complicated than this: https://t.co/Rwkb30OEgq
+Talking to @SquawkStreet about $AAPL &amp; more at @NYSE ! https://t.co/m05b68VLMp
+iPhone sales sour #Apple's earning: https://t.co/962fj9SWsc $AAPL https://t.co/nz9FRK6sNK
+People aren\u2019t upgrading smartphones as quickly and that is bad for Apple https://t.co/EOEJPfNR8Z \U0001f513 $AAPL
+"$NXPI $JBLU $FCX $AAPL $CMG
+$TWTR $EBAY $BWLD $PNRA $CRUS
+$FB $FSLR $UPS $CELG $AMZN
+$LNKD $BIDU $SWKS $GILD $HELE https://t.co/rQUmhHgYn0"
+People mad that Icahn sold $AAPL without giving them the head\u2019s up - How much in commissions did you pay him this year?
+Cool stat: $AAPL's $46 billion loss in market cap overnight is greater than the market cap of 391 S&amp;P 500 companies https://t.co/1ms1YZzTbP
+Apple. You've come a long way... https://t.co/WGvk8K8MYv $AAPL https://t.co/3Wo0hAwRAc
+"Someone is building the Internet's biggest list of stock market cliches and it's amazing: https://t.co/mIV169cF36
+
+$SPY $AAPL $EURUSD"
+JUST IN: Apple delays earnings release by one day, to April 26th after the bell. \u2022 $AAPL
+Apple's market value is down nearly three Twitters $AAPL $TWTR
+Trump warns of a tech bubble: https://t.co/6Ks1yTa4Zc $AAPL $FB $AMZN He's 100% right about this. https://t.co/dJgTLk5JOB
+Apple could sell its billionth iPhone in just a few months' time https://t.co/g6VYDFIE3d $AAPL https://t.co/jzucmxDYXe
+$SPX  KEEP BLOWING #STOCKMARKET #BUBBLE #STOCKS $MSFT $GS $AAPL $SPY $DIA $DJI $C $SIRI $PCLN  $BAC $JPM $VIX $TVIX https://t.co/GPFBb0uCLF
+Will Apple $AAPL fall from tree? 12-mo descending triangle. I've no interest to short it, but it will be wild ride https://t.co/AnjsIKmIHI
+Tim Cook shouldn't be doing TV w/out a new product. Looks desperate. Not a consumer-facing guy. $AAPL https://t.co/Z4UFSimTLg
+When will Apple will sell its billionth iPhone? It may be sooner than you think: https://t.co/5IaF018N1p $AAPL https://t.co/cCIgtKqWHA
+#Stockmarket downtrend continues next week $spx  $spy $vix $tvix $dji $aapl $jpm $bac $c $msft $pcln $wmt $ba https://t.co/1TTlgnKnZc
+$AAPL https://t.co/AFANPYHnoq
+40 years ago this month, Apple co-founder Ronald Wayne sold his 10% stake in $AAPL for $800. Current value: $61 billion.
+Warren Buffett's Berkshire Hathaway reports 9.8M share stake in $AAPL https://t.co/rXWwuyIooI https://t.co/TztgKCcWWy
+Apple's iBooks and iTunes Movies in China have been shut down after less than 7 months https://t.co/ZuGXZqSHma $AAPL https://t.co/1OHGC9YiUf
+Possible buy on $AAPL as it drops onto it's 9 DEMA support #1Broker #Bitcoin #Blockchain https://t.co/WWssD01joh https://t.co/jOKJyG9EaJ
+"Apple is down 7% after earnings.
+
+That's about $40 BILLION in market cap gone in 30 minutes. Poof.
+
+$AAPL: https://t.co/ggfmPjJjkW"
+B4 CRASH 2008 - Paulson's speech:" OUR FINANCIAL SYSTEM IS STRONG" $VXX $VIX $TVIX $UVXY $SPX $SPY $AAPL $MSFT $BAC $C $FB $DJI $DIA $F
+$ONCI is ready to RUN this week! #stockmarket #pennystocks #parabolic $CDNL $MGT $GOOGL $AAPL $TSLA $TWTR $ONCI https://t.co/wwqf0RNOix
+Apple could sell its billionth iPhone in just a few months' time https://t.co/u2qFZ440dH $AAPL https://t.co/8cAchiZ0vC
+The iPhone might radically change in 2017 $AAPL https://t.co/IXLdCfEdus https://t.co/GpdMvFZPjE
+"The growth of smartphones. On one graph.
+
+A great share via: https://t.co/2hAJlarjSM
+
+$AAPL $GOOGL $MSFT https://t.co/BAwQRvYzou"
+"$AAPL finished last quarter with $232 billion in cash, meanwhile Kanye running up debts making records for Tidal. 
+
+Bro."
+Which is bullish for $AAPL if you know anything about $GS https://t.co/WWssD01joh  https://t.co/CQk8iKMI7w
+"The tech stocks with the MOST revenue
+
+1. $AAPL
+2. $AMZN
+3. $MSFT
+
+Visual by @OphirGottlieb: https://t.co/GpZ5ct2z5r https://t.co/H6sNKdtBHd"
+


[6/6] apex-malhar git commit: APEXMALHAR-2142 #comment Implement WindowedStream interface

Posted by th...@apache.org.
APEXMALHAR-2142 #comment Implement WindowedStream interface

Add Accumulation interface support to High-Level API

Add name support for all the windowed transforms

Flex types in Composite transformation

Add more documents and logs


Project: http://git-wip-us.apache.org/repos/asf/apex-malhar/repo
Commit: http://git-wip-us.apache.org/repos/asf/apex-malhar/commit/266b0411
Tree: http://git-wip-us.apache.org/repos/asf/apex-malhar/tree/266b0411
Diff: http://git-wip-us.apache.org/repos/asf/apex-malhar/diff/266b0411

Branch: refs/heads/master
Commit: 266b04116760dbd4d5cad6b4102b06153ac96a5f
Parents: 17f6c55
Author: Siyuan Hua <hs...@apache.org>
Authored: Tue Jul 12 11:57:09 2016 -0700
Committer: Siyuan Hua <hs...@apache.org>
Committed: Thu Aug 25 09:26:03 2016 -0700

----------------------------------------------------------------------
 demos/highlevelapi/pom.xml                      |  55 ++
 demos/highlevelapi/src/assemble/appPackage.xml  |  59 ++
 .../sample/cookbook/CombinePerKeyExamples.java  | 237 ++++++++
 .../stream/sample/cookbook/TriggerExample.java  | 578 +++++++++++++++++++
 demos/pom.xml                                   |  12 +
 stream/pom.xml                                  |   1 -
 .../apex/malhar/stream/api/ApexStream.java      | 144 ++---
 .../stream/api/CompositeStreamTransform.java    |  30 +
 .../apache/apex/malhar/stream/api/Option.java   | 122 ++++
 .../apex/malhar/stream/api/WindowedStream.java  | 150 +++++
 .../malhar/stream/api/function/Function.java    |  36 +-
 .../malhar/stream/api/impl/ApexStreamImpl.java  | 194 +++----
 .../stream/api/impl/ApexWindowedStreamImpl.java | 275 +++++++++
 .../apex/malhar/stream/api/impl/DagMeta.java    |  53 +-
 .../malhar/stream/api/impl/IDGenerator.java     |   3 +
 .../malhar/stream/api/impl/StreamFactory.java   |  76 ++-
 .../stream/api/impl/TupleWrapperOperator.java   | 192 ++++++
 .../stream/api/impl/accumulation/Count.java     |  61 ++
 .../stream/api/impl/accumulation/FoldFn.java    |  65 +++
 .../stream/api/impl/accumulation/ReduceFn.java  |  65 +++
 .../stream/api/impl/accumulation/TopN.java      | 107 ++++
 .../api/operator/AnnonymousClassModifier.java   |   3 +
 .../api/operator/ByteArrayClassLoader.java      |   3 +
 .../stream/api/operator/FunctionOperator.java   | 112 ++--
 .../apex/malhar/stream/api/util/KeyedTuple.java |  34 --
 .../apex/malhar/stream/api/util/TupleUtil.java  |  21 +-
 .../FunctionOperator/FunctionOperatorTest.java  |   4 +-
 .../stream/api/impl/ApexStreamImplTest.java     |   6 +-
 .../stream/sample/ApplicationWithStreamAPI.java |  21 +-
 .../LocalTestWithoutStreamApplication.java      |  54 +-
 .../apex/malhar/stream/sample/MyStream.java     |   3 +-
 .../apex/malhar/stream/sample/MyStreamTest.java |  87 ++-
 .../malhar/stream/sample/TupleCollector.java    |   2 +
 .../apex/malhar/stream/sample/WCInput.java      |  90 +++
 .../stream/sample/WordCountWithStreamAPI.java   |  72 +++
 stream/src/test/resources/sampletweets.txt      | 207 +++++++
 36 files changed, 2839 insertions(+), 395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/demos/highlevelapi/pom.xml
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/pom.xml b/demos/highlevelapi/pom.xml
new file mode 100644
index 0000000..c669681
--- /dev/null
+++ b/demos/highlevelapi/pom.xml
@@ -0,0 +1,55 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>high-level-api-demo</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Apache Apex Malhar High-Level API Demo</name>
+  <description>Apex demo applications that use High-level API to construct a dag</description>
+
+  <parent>
+    <groupId>org.apache.apex</groupId>
+    <artifactId>malhar-demos</artifactId>
+    <version>3.5.0-SNAPSHOT</version>
+  </parent>
+
+  <properties>
+    <skipTests>true</skipTests>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>cglib</groupId>
+      <artifactId>cglib</artifactId>
+      <version>3.2.1</version>
+    </dependency>
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <version>2.9.1</version>
+    </dependency>
+  </dependencies>
+
+
+</project>

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/demos/highlevelapi/src/assemble/appPackage.xml
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/assemble/appPackage.xml b/demos/highlevelapi/src/assemble/appPackage.xml
new file mode 100644
index 0000000..4138cf2
--- /dev/null
+++ b/demos/highlevelapi/src/assemble/appPackage.xml
@@ -0,0 +1,59 @@
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+  <id>appPackage</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <fileSet>
+      <directory>${basedir}/target/</directory>
+      <outputDirectory>/app</outputDirectory>
+      <includes>
+        <include>${project.artifactId}-${project.version}.jar</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>${basedir}/target/deps</directory>
+      <outputDirectory>/lib</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>${basedir}/src/site/conf</directory>
+      <outputDirectory>/conf</outputDirectory>
+      <includes>
+        <include>*.xml</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>${basedir}/src/main/resources/META-INF</directory>
+      <outputDirectory>/META-INF</outputDirectory>
+    </fileSet>
+    <fileSet>
+      <directory>${basedir}/src/main/resources/app</directory>
+      <outputDirectory>/app</outputDirectory>
+    </fileSet>
+  </fileSets>
+
+</assembly>
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java
new file mode 100644
index 0000000..5d4c628
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/CombinePerKeyExamples.java
@@ -0,0 +1,237 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.cookbook;
+
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.apex.malhar.stream.api.impl.accumulation.ReduceFn;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.lib.util.KeyValPair;
+
+/**
+ * An example that reads the public 'Shakespeare' data, and for each word in
+ * the dataset that is over a given length, generates a string containing the
+ * list of play names in which that word appears
+ *
+ * <p>Concepts: the combine transform, which lets you combine the values in a
+ * key-grouped Collection
+ *
+ */
+public class CombinePerKeyExamples
+{
+  // Use the shakespeare public BigQuery sample
+  private static final String SHAKESPEARE_TABLE = "publicdata:samples.shakespeare";
+  // We'll track words >= this word length across all plays in the table.
+  private static final int MIN_WORD_LENGTH = 9;
+
+  /**
+   * Examines each row in the input table. If the word is greater than or equal to MIN_WORD_LENGTH,
+   * outputs word, play_name.
+   */
+  static class ExtractLargeWordsFn implements Function.MapFunction<SampleBean, KeyValPair<String, String>>
+  {
+
+    @Override
+    public KeyValPair<String, String> f(SampleBean input)
+    {
+      String playName = input.getCorpus();
+      String word = input.getWord();
+      if (word.length() >= MIN_WORD_LENGTH) {
+        return new KeyValPair<>(word, playName);
+      } else {
+        return null;
+      }
+    }
+  }
+
+
+  /**
+   * Prepares the output data which is in same bean
+   */
+  static class FormatShakespeareOutputFn implements Function.MapFunction<Tuple.WindowedTuple<KeyValPair<String, String>>, SampleBean>
+  {
+    @Override
+    public SampleBean f(Tuple.WindowedTuple<KeyValPair<String, String>> input)
+    {
+      return new SampleBean(input.getValue().getKey(), input.getValue().getValue(), null);
+    }
+  }
+
+  /**
+   * Reads the public 'Shakespeare' data, and for each word in the dataset
+   * over a given length, generates a string containing the list of play names
+   * in which that word appears.
+   */
+  static class PlaysForWord
+      extends CompositeStreamTransform<SampleBean, SampleBean>
+  {
+
+    @Override
+    public ApexStream<SampleBean> compose(ApexStream<SampleBean> inputStream)
+    {
+      // fix this later
+      return inputStream.map(new ExtractLargeWordsFn())
+          .window(new WindowOption.GlobalWindow())
+          .reduceByKey(new ReduceFn<String>()
+          {
+            @Override
+            public String defaultAccumulatedValue()
+            {
+              return "";
+            }
+
+            @Override
+            public String accumulate(String accumulatedValue, String input)
+            {
+              return accumulatedValue + "," + input;
+            }
+
+            @Override
+            public String merge(String accumulatedValue1, String accumulatedValue2)
+            {
+              return accumulatedValue1 + "," + accumulatedValue2;
+            }
+
+            @Override
+            public String getOutput(String accumulatedValue)
+            {
+              return accumulatedValue;
+            }
+
+            @Override
+            public String getRetraction(String value)
+            {
+              return value;
+            }
+          }, new Function.MapFunction<KeyValPair<String, String>, Tuple<KeyValPair<String, String>>>()
+
+          {
+            @Override
+            public Tuple<KeyValPair<String, String>> f(KeyValPair<String, String> input)
+            {
+              return null;
+            }
+          })
+          .map(new FormatShakespeareOutputFn());
+    }
+  }
+
+
+  public static class SampleBean
+  {
+
+    public SampleBean()
+    {
+
+    }
+
+    public SampleBean(String word, String all_plays, String corpus)
+    {
+      this.word = word;
+      this.all_plays = all_plays;
+      this.corpus = corpus;
+    }
+
+    private String word;
+
+    private String all_plays;
+
+    private String corpus;
+
+    public void setWord(String word)
+    {
+      this.word = word;
+    }
+
+    public String getWord()
+    {
+      return word;
+    }
+
+    public void setCorpus(String corpus)
+    {
+      this.corpus = corpus;
+    }
+
+    public String getCorpus()
+    {
+      return corpus;
+    }
+
+    public void setAll_plays(String all_plays)
+    {
+      this.all_plays = all_plays;
+    }
+
+    public String getAll_plays()
+    {
+      return all_plays;
+    }
+  }
+
+  public static class SampleInput implements InputOperator
+  {
+
+    public final transient DefaultOutputPort<SampleBean> beanOutput = new DefaultOutputPort();
+
+    @Override
+    public void emitTuples()
+    {
+
+    }
+
+    @Override
+    public void beginWindow(long l)
+    {
+
+    }
+
+    @Override
+    public void endWindow()
+    {
+
+    }
+
+    @Override
+    public void setup(Context.OperatorContext context)
+    {
+
+    }
+
+    @Override
+    public void teardown()
+    {
+
+    }
+  }
+
+
+  public static void main(String[] args) throws Exception
+  {
+    SampleInput input = new SampleInput();
+    StreamFactory.fromInput(input, input.beanOutput).addCompositeStreams(new PlaysForWord());
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java
new file mode 100644
index 0000000..903f624
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/cookbook/TriggerExample.java
@@ -0,0 +1,578 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.cookbook;
+
+import java.util.Date;
+import java.util.Objects;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+
+import com.datatorrent.lib.util.KeyValPair;
+
+/**
+ * This example illustrates the basic concepts behind triggering. It shows how to use different
+ * trigger definitions to produce partial (speculative) results before all the data is processed and
+ * to control when updated results are produced for late data. The example performs a streaming
+ * analysis of the data coming in from PubSub and writes the results to BigQuery. It divides the
+ * data into {@link Window windows} to be processed, and demonstrates using various kinds of
+ * {@link org.apache.beam.sdk.transforms.windowing.Trigger triggers} to control when the results for
+ * each window are emitted.
+ *
+ * <p> This example uses a portion of real traffic data from San Diego freeways. It contains
+ * readings from sensor stations set up along each freeway. Each sensor reading includes a
+ * calculation of the 'total flow' across all lanes in that freeway direction.
+ *
+ * <p> Concepts:
+ * <pre>
+ *   1. The default triggering behavior
+ *   2. Late data with the default trigger
+ *   3. How to get speculative estimates
+ *   4. Combining late data and speculative estimates
+ * </pre>
+ *
+ * <p> Before running this example, it will be useful to familiarize yourself with Dataflow triggers
+ * and understand the concept of 'late data',
+ * See:  <a href="https://cloud.google.com/dataflow/model/triggers">
+ * https://cloud.google.com/dataflow/model/triggers </a> and
+ * <a href="https://cloud.google.com/dataflow/model/windowing#Advanced">
+ * https://cloud.google.com/dataflow/model/windowing#Advanced </a>
+ *
+ * <p> The example pipeline reads data from a Pub/Sub topic. By default, running the example will
+ * also run an auxiliary pipeline to inject data from the default {@code --input} file to the
+ * {@code --pubsubTopic}. The auxiliary pipeline puts a timestamp on the injected data so that the
+ * example pipeline can operate on <i>event time</i> (rather than arrival time). The auxiliary
+ * pipeline also randomly simulates late data, by setting the timestamps of some of the data
+ * elements to be in the past. You may override the default {@code --input} with the file of your
+ * choosing or set {@code --input=""} which will disable the automatic Pub/Sub injection, and allow
+ * you to use a separate tool to publish to the given topic.
+ *
+ * <p> The example is configured to use the default Pub/Sub topic and the default BigQuery table
+ * from the example common package (there are no defaults for a general Dataflow pipeline).
+ * You can override them by using the {@code --pubsubTopic}, {@code --bigQueryDataset}, and
+ * {@code --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist,
+ * the example will try to create them.
+ *
+ * <p> The pipeline outputs its results to a BigQuery table.
+ * Here are some queries you can use to see interesting results:
+ * Replace {@code <enter_table_name>} in the query below with the name of the BigQuery table.
+ * Replace {@code <enter_window_interval>} in the query below with the window interval.
+ *
+ * <p> To see the results of the default trigger,
+ * Note: When you start up your pipeline, you'll initially see results from 'late' data. Wait after
+ * the window duration, until the first pane of non-late data has been emitted, to see more
+ * interesting results.
+ * {@code SELECT * FROM enter_table_name WHERE trigger_type = "default" ORDER BY window DESC}
+ *
+ * <p> To see the late data i.e. dropped by the default trigger,
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "withAllowedLateness" and
+ * (timing = "LATE" or timing = "ON_TIME") and freeway = "5" ORDER BY window DESC, processing_time}
+ *
+ * <p>To see the the difference between accumulation mode and discarding mode,
+ * {@code SELECT * FROM <enter_table_name> WHERE (timing = "LATE" or timing = "ON_TIME") AND
+ * (trigger_type = "withAllowedLateness" or trigger_type = "sequential") and freeway = "5" ORDER BY
+ * window DESC, processing_time}
+ *
+ * <p> To see speculative results every minute,
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "speculative" and freeway = "5"
+ * ORDER BY window DESC, processing_time}
+ *
+ * <p> To see speculative results every five minutes after the end of the window
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "sequential" and timing != "EARLY"
+ * and freeway = "5" ORDER BY window DESC, processing_time}
+ *
+ * <p> To see the first and the last pane for a freeway in a window for all the trigger types,
+ * {@code SELECT * FROM <enter_table_name> WHERE (isFirst = true or isLast = true) ORDER BY window}
+ *
+ * <p> To reduce the number of results for each query we can add additional where clauses.
+ * For examples, To see the results of the default trigger,
+ * {@code SELECT * FROM <enter_table_name> WHERE trigger_type = "default" AND freeway = "5" AND
+ * window = "<enter_window_interval>"}
+ *
+ * <p> The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
+ * and then exits.
+ */
+
+public class TriggerExample
+{
+  //Numeric value of fixed window duration, in minutes
+  public static final int WINDOW_DURATION = 30;
+  // Constants used in triggers.
+  // Speeding up ONE_MINUTE or FIVE_MINUTES helps you get an early approximation of results.
+  // ONE_MINUTE is used only with processing time before the end of the window
+  public static final Duration ONE_MINUTE = Duration.standardMinutes(1);
+  // FIVE_MINUTES is used only with processing time after the end of the window
+  public static final Duration FIVE_MINUTES = Duration.standardMinutes(5);
+  // ONE_DAY is used to specify the amount of lateness allowed for the data elements.
+  public static final Duration ONE_DAY = Duration.standardDays(1);
+
+  /**
+   * This transform demonstrates using triggers to control when data is produced for each window
+   * Consider an example to understand the results generated by each type of trigger.
+   * The example uses "freeway" as the key. Event time is the timestamp associated with the data
+   * element and processing time is the time when the data element gets processed in the pipeline.
+   * For freeway 5, suppose there are 10 elements in the [10:00:00, 10:30:00) window.
+   * Key (freeway) | Value (total_flow) | event time | processing time
+   * 5             | 50                 | 10:00:03   | 10:00:47
+   * 5             | 30                 | 10:01:00   | 10:01:03
+   * 5             | 30                 | 10:02:00   | 11:07:00
+   * 5             | 20                 | 10:04:10   | 10:05:15
+   * 5             | 60                 | 10:05:00   | 11:03:00
+   * 5             | 20                 | 10:05:01   | 11.07:30
+   * 5             | 60                 | 10:15:00   | 10:27:15
+   * 5             | 40                 | 10:26:40   | 10:26:43
+   * 5             | 60                 | 10:27:20   | 10:27:25
+   * 5             | 60                 | 10:29:00   | 11:11:00
+   *
+   * <p> Dataflow tracks a watermark which records up to what point in event time the data is
+   * complete. For the purposes of the example, we'll assume the watermark is approximately 15m
+   * behind the current processing time. In practice, the actual value would vary over time based
+   * on the systems knowledge of the current PubSub delay and contents of the backlog (data
+   * that has not yet been processed).
+   *
+   * <p> If the watermark is 15m behind, then the window [10:00:00, 10:30:00) (in event time) would
+   * close at 10:44:59, when the watermark passes 10:30:00.
+   */
+  static class CalculateTotalFlow
+      extends CompositeStreamTransform<String, SampleBean>
+  {
+    private int windowDuration;
+
+    CalculateTotalFlow(int windowDuration)
+    {
+      this.windowDuration = windowDuration;
+    }
+
+    @Override
+    public ApexStream<SampleBean> compose(ApexStream<String> inputStream)
+    {
+      // Concept #1: The default triggering behavior
+      // By default Dataflow uses a trigger which fires when the watermark has passed the end of the
+      // window. This would be written {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
+
+      // The system also defaults to dropping late data -- data which arrives after the watermark
+      // has passed the event timestamp of the arriving element. This means that the default trigger
+      // will only fire once.
+
+      // Each pane produced by the default trigger with no allowed lateness will be the first and
+      // last pane in the window, and will be ON_TIME.
+
+      // The results for the example above with the default trigger and zero allowed lateness
+      // would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 260                | 6                 | true    | true   | ON_TIME
+
+      // At 11:03:00 (processing time) the system watermark may have advanced to 10:54:00. As a
+      // result, when the data record with event time 10:05:00 arrives at 11:03:00, it is considered
+      // late, and dropped.
+
+      ApexStream<SampleBean> defaultTriggerResults = inputStream
+          .window(new WindowOption.TimeWindows(Duration.standardMinutes(windowDuration)),
+          new TriggerOption().discardingFiredPanes())
+          .addCompositeStreams(new TotalFlow("default"));
+
+      // Concept #2: Late data with the default trigger
+      // This uses the same trigger as concept #1, but allows data that is up to ONE_DAY late. This
+      // leads to each window staying open for ONE_DAY after the watermark has passed the end of the
+      // window. Any late data will result in an additional pane being fired for that same window.
+
+      // The first pane produced will be ON_TIME and the remaining panes will be LATE.
+      // To definitely get the last pane when the window closes, use
+      // .withAllowedLateness(ONE_DAY, ClosingBehavior.FIRE_ALWAYS).
+
+      // The results for the example above with the default trigger and ONE_DAY allowed lateness
+      // would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 260                | 6                 | true    | false  | ON_TIME
+      // 5             | 60                 | 1                 | false   | false  | LATE
+      // 5             | 30                 | 1                 | false   | false  | LATE
+      // 5             | 20                 | 1                 | false   | false  | LATE
+      // 5             | 60                 | 1                 | false   | false  | LATE
+      ApexStream<SampleBean> withAllowedLatenessResults = inputStream
+          .window(new WindowOption.TimeWindows(Duration.standardMinutes(windowDuration)),
+          new TriggerOption().discardingFiredPanes(),
+          Duration.standardDays(1))
+          .addCompositeStreams(new TotalFlow("withAllowedLateness"));
+
+      // Concept #3: How to get speculative estimates
+      // We can specify a trigger that fires independent of the watermark, for instance after
+      // ONE_MINUTE of processing time. This allows us to produce speculative estimates before
+      // all the data is available. Since we don't have any triggers that depend on the watermark
+      // we don't get an ON_TIME firing. Instead, all panes are either EARLY or LATE.
+
+      // We also use accumulatingFiredPanes to build up the results across each pane firing.
+
+      // The results for the example above for this trigger would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 80                 | 2                 | true    | false  | EARLY
+      // 5             | 100                | 3                 | false   | false  | EARLY
+      // 5             | 260                | 6                 | false   | false  | EARLY
+      // 5             | 320                | 7                 | false   | false  | LATE
+      // 5             | 370                | 9                 | false   | false  | LATE
+      // 5             | 430                | 10                | false   | false  | LATE
+
+      ApexStream<SampleBean> speculativeResults = inputStream
+          .window(new WindowOption.TimeWindows(Duration.standardMinutes(windowDuration)),
+              //Trigger fires every minute
+          new TriggerOption().withEarlyFiringsAtEvery(Duration.standardMinutes(1))
+                  // After emitting each pane, it will continue accumulating the elements so that each
+                  // approximation includes all of the previous data in addition to the newly arrived
+                  // data.
+          .accumulatingFiredPanes(),
+          Duration.standardDays(1))
+          .addCompositeStreams(new TotalFlow("speculative"));
+
+      // Concept #4: Combining late data and speculative estimates
+      // We can put the previous concepts together to get EARLY estimates, an ON_TIME result,
+      // and LATE updates based on late data.
+
+      // Each time a triggering condition is satisfied it advances to the next trigger.
+      // If there are new elements this trigger emits a window under following condition:
+      // > Early approximations every minute till the end of the window.
+      // > An on-time firing when the watermark has passed the end of the window
+      // > Every five minutes of late data.
+
+      // Every pane produced will either be EARLY, ON_TIME or LATE.
+
+      // The results for the example above for this trigger would be:
+      // Key (freeway) | Value (total_flow) | number_of_records | isFirst | isLast | timing
+      // 5             | 80                 | 2                 | true    | false  | EARLY
+      // 5             | 100                | 3                 | false   | false  | EARLY
+      // 5             | 260                | 6                 | false   | false  | EARLY
+      // [First pane fired after the end of the window]
+      // 5             | 320                | 7                 | false   | false  | ON_TIME
+      // 5             | 430                | 10                | false   | false  | LATE
+
+      // For more possibilities of how to build advanced triggers, see {@link Trigger}.
+      ApexStream<SampleBean> sequentialResults = inputStream
+          .window(new WindowOption.TimeWindows(Duration.standardMinutes(windowDuration)),
+              // Speculative every ONE_MINUTE
+          new TriggerOption().withEarlyFiringsAtEvery(Duration.standardMinutes(1))
+          .withLateFiringsAtEvery(Duration.standardMinutes(5))
+                  // After emitting each pane, it will continue accumulating the elements so that each
+                  // approximation includes all of the previous data in addition to the newly arrived
+                  // data.
+          .accumulatingFiredPanes(),
+          Duration.standardDays(1))
+          .addCompositeStreams(new TotalFlow("sequential"));
+
+      return sequentialResults;
+    }
+
+  }
+
+  //////////////////////////////////////////////////////////////////////////////////////////////////
+  // The remaining parts of the pipeline are needed to produce the output for each
+  // concept above. Not directly relevant to understanding the trigger examples.
+
+  /**
+   * Calculate total flow and number of records for each freeway and format the results to TableRow
+   * objects, to save to BigQuery.
+   */
+  static class TotalFlow extends
+      CompositeStreamTransform<String, SampleBean>
+  {
+    private String triggerType;
+
+    public TotalFlow(String triggerType)
+    {
+      this.triggerType = triggerType;
+    }
+
+    @Override
+    public ApexStream<SampleBean> compose(ApexStream<String> inputStream)
+    {
+      if (!(inputStream instanceof WindowedStream)) {
+        throw new RuntimeException("Not supported here");
+      }
+      WindowedStream<String> windowedStream = (WindowedStream<String>)inputStream;
+      ApexStream<KeyValPair<String, Iterable<Integer>>> flowPerFreeway = windowedStream
+          .groupByKey(new ExtractFlowInfo());
+
+      return flowPerFreeway
+          .map(new Function.MapFunction<KeyValPair<String, Iterable<Integer>>, KeyValPair<String, String>>()
+          {
+            @Override
+            public KeyValPair<String, String> f(KeyValPair<String, Iterable<Integer>> input)
+            {
+              Iterable<Integer> flows = input.getValue();
+              Integer sum = 0;
+              Long numberOfRecords = 0L;
+              for (Integer value : flows) {
+                sum += value;
+                numberOfRecords++;
+              }
+              return new KeyValPair<>(input.getKey(), sum + "," + numberOfRecords);
+            }
+          })
+          .map(new FormatTotalFlow(triggerType));
+    }
+  }
+
+  /**
+   * Format the results of the Total flow calculation to a TableRow, to save to BigQuery.
+   * Adds the triggerType, pane information, processing time and the window timestamp.
+   */
+  static class FormatTotalFlow implements Function.MapFunction<KeyValPair<String, String>, SampleBean>
+  {
+    private String triggerType;
+
+    public FormatTotalFlow(String triggerType)
+    {
+      this.triggerType = triggerType;
+    }
+
+    @Override
+    public SampleBean f(KeyValPair<String, String> input)
+    {
+      String[] values = input.getValue().split(",");
+      //TODO need to have a callback to get the metadata like window id, pane id, timestamps etc.
+      return new SampleBean(triggerType, input.getKey(), Integer.parseInt(values[0]), Long
+          .parseLong(values[1]), null, false, false, null, null, new Date());
+    }
+  }
+
+  public static class SampleBean
+  {
+    public SampleBean()
+    {
+    }
+
+    private String trigger_type;
+
+    private String freeway;
+
+    private int total_flow;
+
+    private long number_of_records;
+
+    private String window;
+
+    private boolean isFirst;
+
+    private boolean isLast;
+
+    private Date timing;
+
+    private Date event_time;
+
+    private Date processing_time;
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      SampleBean that = (SampleBean)o;
+      return total_flow == that.total_flow &&
+          number_of_records == that.number_of_records &&
+          isFirst == that.isFirst &&
+          isLast == that.isLast &&
+          Objects.equals(trigger_type, that.trigger_type) &&
+          Objects.equals(freeway, that.freeway) &&
+          Objects.equals(window, that.window) &&
+          Objects.equals(timing, that.timing) &&
+          Objects.equals(event_time, that.event_time) &&
+          Objects.equals(processing_time, that.processing_time);
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects
+          .hash(trigger_type, freeway, total_flow, number_of_records, window, isFirst, isLast, timing, event_time,
+              processing_time);
+    }
+
+    public SampleBean(String trigger_type, String freeway, int total_flow, long number_of_records, String window,
+        boolean isFirst, boolean isLast, Date timing, Date event_time, Date processing_time)
+    {
+
+      this.trigger_type = trigger_type;
+      this.freeway = freeway;
+      this.total_flow = total_flow;
+      this.number_of_records = number_of_records;
+      this.window = window;
+      this.isFirst = isFirst;
+      this.isLast = isLast;
+      this.timing = timing;
+      this.event_time = event_time;
+      this.processing_time = processing_time;
+    }
+
+    public String getTrigger_type()
+    {
+      return trigger_type;
+    }
+
+    public void setTrigger_type(String trigger_type)
+    {
+      this.trigger_type = trigger_type;
+    }
+
+    public String getFreeway()
+    {
+      return freeway;
+    }
+
+    public void setFreeway(String freeway)
+    {
+      this.freeway = freeway;
+    }
+
+    public int getTotal_flow()
+    {
+      return total_flow;
+    }
+
+    public void setTotal_flow(int total_flow)
+    {
+      this.total_flow = total_flow;
+    }
+
+    public long getNumber_of_records()
+    {
+      return number_of_records;
+    }
+
+    public void setNumber_of_records(long number_of_records)
+    {
+      this.number_of_records = number_of_records;
+    }
+
+    public String getWindow()
+    {
+      return window;
+    }
+
+    public void setWindow(String window)
+    {
+      this.window = window;
+    }
+
+    public boolean isFirst()
+    {
+      return isFirst;
+    }
+
+    public void setFirst(boolean first)
+    {
+      isFirst = first;
+    }
+
+    public boolean isLast()
+    {
+      return isLast;
+    }
+
+    public void setLast(boolean last)
+    {
+      isLast = last;
+    }
+
+    public Date getTiming()
+    {
+      return timing;
+    }
+
+    public void setTiming(Date timing)
+    {
+      this.timing = timing;
+    }
+
+    public Date getEvent_time()
+    {
+      return event_time;
+    }
+
+    public void setEvent_time(Date event_time)
+    {
+      this.event_time = event_time;
+    }
+
+    public Date getProcessing_time()
+    {
+      return processing_time;
+    }
+
+    public void setProcessing_time(Date processing_time)
+    {
+      this.processing_time = processing_time;
+    }
+  }
+
+  /**
+   * Extract the freeway and total flow in a reading.
+   * Freeway is used as key since we are calculating the total flow for each freeway.
+   */
+  static class ExtractFlowInfo implements Function.MapFunction<String, KeyValPair<String, Integer>>
+  {
+    @Override
+    public KeyValPair<String, Integer> f(String input)
+    {
+      String[] laneInfo = input.split(",");
+      if (laneInfo[0].equals("timestamp")) {
+        // Header row
+        return null;
+      }
+      if (laneInfo.length < 48) {
+        //Skip the invalid input.
+        return null;
+      }
+      String freeway = laneInfo[2];
+      Integer totalFlow = tryIntegerParse(laneInfo[7]);
+      // Ignore the records with total flow 0 to easily understand the working of triggers.
+      // Skip the records with total flow -1 since they are invalid input.
+      if (totalFlow == null || totalFlow <= 0) {
+        return null;
+      }
+      return new KeyValPair<>(freeway, totalFlow);
+    }
+  }
+
+  private static final String PUBSUB_TIMESTAMP_LABEL_KEY = "timestamp_ms";
+
+  public static void main(String[] args) throws Exception
+  {
+    StreamFactory.fromFolder("some folder")
+        .addCompositeStreams(new CalculateTotalFlow(60));
+
+  }
+
+  private static Integer tryIntegerParse(String number)
+  {
+    try {
+      return Integer.parseInt(number);
+    } catch (NumberFormatException e) {
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/demos/pom.xml
----------------------------------------------------------------------
diff --git a/demos/pom.xml b/demos/pom.xml
index 12f0f14..e9f2daf 100644
--- a/demos/pom.xml
+++ b/demos/pom.xml
@@ -192,6 +192,7 @@
     <module>r</module>
     <module>echoserver</module>
     <module>iteration</module>
+    <module>highlevelapi</module>
   </modules>
 
   <dependencies>
@@ -231,6 +232,17 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.apex</groupId>
+      <artifactId>malhar-stream</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/pom.xml
----------------------------------------------------------------------
diff --git a/stream/pom.xml b/stream/pom.xml
index fd663e0..445be92 100755
--- a/stream/pom.xml
+++ b/stream/pom.xml
@@ -95,6 +95,5 @@
       <version>3.2.1</version>
     </dependency>
 
-
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java
index 2f65ba9..6d44534 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/ApexStream.java
@@ -18,11 +18,14 @@
  */
 package org.apache.apex.malhar.stream.api;
 
-
-import java.util.Map;
 import java.util.concurrent.Callable;
 
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.WindowOption;
 import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.hadoop.classification.InterfaceStability;
 
 import com.datatorrent.api.Attribute;
 import com.datatorrent.api.Context.DAGContext;
@@ -37,6 +40,7 @@ import com.datatorrent.api.Operator;
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public interface ApexStream<T>
 {
   /**
@@ -46,17 +50,7 @@ public interface ApexStream<T>
    * @param <O> Type of the output
    * @return new stream of type O
    */
-  <O, STREAM extends ApexStream<O>> STREAM map(Function.MapFunction<T, O> mapFunction);
-
-  /**
-   * Simple map transformation<br>
-   * Add an operator to the DAG which convert tuple T to tuple O
-   * @param name operator name
-   * @param mapFunction map function
-   * @param <O> Type of the output
-   * @return new stream of type O
-   */
-  <O, STREAM extends ApexStream<O>> STREAM map(String name, Function.MapFunction<T, O> mapFunction);
+  <O, STREAM extends ApexStream<O>> STREAM map(Function.MapFunction<T, O> mapFunction, Option... opts);
 
   /**
    * Flat map transformation
@@ -65,17 +59,7 @@ public interface ApexStream<T>
    * @param <O> Type of the output
    * @return new stream of type O
    */
-  <O, STREAM extends ApexStream<O>> STREAM flatMap(Function.FlatMapFunction<T, O> flatten);
-
-  /**
-   * Flat map transformation<br>
-   * Add an operator to the DAG which convert tuple T to a collection of tuple O
-   * @param name operator name
-   * @param flatten
-   * @param <O> Type of the output
-   * @return new stream of type O
-   */
-  <O, STREAM extends ApexStream<O>> STREAM flatMap(String name, Function.FlatMapFunction<T, O> flatten);
+  <O, STREAM extends ApexStream<O>> STREAM flatMap(Function.FlatMapFunction<T, O> flatten, Option... opts);
 
   /**
    * Filter transformation<br>
@@ -83,76 +67,7 @@ public interface ApexStream<T>
    * @param filter filter function
    * @return new stream of same type
    */
-  <STREAM extends ApexStream<T>> STREAM filter(Function.FilterFunction<T> filter);
-
-  /**
-   * Filter transformation<br>
-   * Add an operator to the DAG which filter out tuple T that cannot satisfy the FilterFunction
-   * @param name operator name
-   * @param filter filter function
-   * @return new stream of same type
-   */
-  <STREAM extends ApexStream<T>> STREAM filter(String name, Function.FilterFunction<T> filter);
-
-  /**
-   * Reduce transformation<br>
-   * Add an operator to the DAG which merge tuple t1, t2 to new tuple
-   * @param reduce reduce function
-   * @return new stream of same type
-   */
-  <STREAM extends ApexStream<T>> STREAM reduce(Function.ReduceFunction<T> reduce);
-
-  /**
-   * Reduce transformation<br>
-   * Add an operator to the DAG which merge tuple t1, t2 to new tuple
-   * @param name operator name
-   * @param reduce reduce function
-   * @return new stream of same type
-   */
-  <STREAM extends ApexStream<T>> STREAM reduce(String name, Function.ReduceFunction<T> reduce);
-
-  /**
-   * Fold transformation<br>
-   * Add an operator to the DAG which merge tuple T to accumulated result tuple O
-   * @param initialValue initial result value
-   * @param fold fold function
-   * @param <O> Result type
-   * @return new stream of type O
-   */
-  <O, STREAM extends ApexStream<O>> STREAM fold(O initialValue, Function.FoldFunction<T, O> fold);
-
-  /**
-   * Fold transformation<br>
-   * Add an operator to the DAG which merge tuple T to accumulated result tuple O
-   * @param name name of the operator
-   * @param initialValue initial result value
-   * @param fold fold function
-   * @param <O> Result type
-   * @return new stream of type O
-   */
-  <O, STREAM extends ApexStream<O>> STREAM fold(String name, O initialValue, Function.FoldFunction<T, O> fold);
-
-  /**
-   * Count of all tuples
-   * @return new stream of Integer
-   */
-  <STREAM extends ApexStream<Integer>> STREAM count();
-
-  /**
-   * Count tuples by the key<br>
-   * If the input is KeyedTuple it will get the key from getKey method from the tuple<br>
-   * If not, use the tuple itself as a key
-   * @return new stream of Map
-   */
-  <STREAM extends ApexStream<Map<Object, Integer>>> STREAM countByKey();
-
-  /**
-   *
-   * Count tuples by the indexed key
-   * @param key the index of the field in the tuple that are used as key
-   * @return new stream of Map
-   */
-  <STREAM extends ApexStream<Map<Object, Integer>>> STREAM countByKey(int key);
+  <STREAM extends ApexStream<T>> STREAM filter(Function.FilterFunction<T> filter, Option... opts);
 
   /**
    * Extend the dag by adding one operator<br>
@@ -162,18 +77,23 @@ public interface ApexStream<T>
    * @param <O> type of the output
    * @return new stream of type O
    */
-  <O, STREAM extends ApexStream<O>> STREAM addOperator(Operator op, Operator.InputPort<T> inputPort,  Operator.OutputPort<O> outputPort);
+  <O, STREAM extends ApexStream<O>> STREAM addOperator(Operator op, Operator.InputPort<T> inputPort,  Operator.OutputPort<O> outputPort, Option... opts);
 
   /**
-   * Extend the dag by adding one {@see Operator}
-   * @param opName Operator name
+   * Extend the dag by adding one end operator<br>
    * @param op Operator added to the stream
    * @param inputPort InputPort of the operator that is connected to last exposed OutputPort in the stream
-   * @param outputPort OutputPort of the operator will be connected to next operator
    * @param <O> type of the output
    * @return new stream of type O
    */
-  <O, STREAM extends ApexStream<O>> STREAM addOperator(String opName, Operator op, Operator.InputPort<T> inputPort,  Operator.OutputPort<O> outputPort);
+  <O, STREAM extends ApexStream<O>> STREAM endWith(Operator op, Operator.InputPort<T> inputPort, Option... opts);
+
+  /**
+   * Extend the dag by adding one {@see CompositeStreamTransform}
+   * @param compositeStreamTransform Composite Streams and Transforms
+   * @return new stream of type O
+   */
+  <O, INSTREAM extends ApexStream<T>, OUTSTREAM extends ApexStream<O>> OUTSTREAM addCompositeStreams(CompositeStreamTransform<INSTREAM, OUTSTREAM> compositeStreamTransform);
 
   /**
    * Union multiple stream into one
@@ -260,4 +180,30 @@ public interface ApexStream<T>
    */
   void run();
 
+  /**
+   * Chunk tuples into Windows
+   * Window Transform are defined in {@see WindowedStream}
+   * @param windowOption
+   * @return
+   */
+  WindowedStream<T> window(WindowOption windowOption);
+
+  /**
+   * Chunk tuple into windows with window option and trigger option
+   * @param windowOption
+   * @param triggerOption
+   * @return
+   */
+  WindowedStream<T> window(WindowOption windowOption, TriggerOption triggerOption);
+
+  /**
+   *
+   * Chunk tuple into windows with window option and trigger option and allowed lateness
+   * @param windowOption
+   * @param triggerOption
+   * @param allowLateness
+   * @return
+   */
+  WindowedStream<T> window(WindowOption windowOption, TriggerOption triggerOption, Duration allowLateness);
+
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/CompositeStreamTransform.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/CompositeStreamTransform.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/CompositeStreamTransform.java
new file mode 100644
index 0000000..979f44f
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/CompositeStreamTransform.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A group of Streams and transforms in between
+ */
+@InterfaceStability.Evolving
+public abstract class CompositeStreamTransform<INSTREAM extends ApexStream, OUTSTREAM extends ApexStream>
+{
+  public abstract OUTSTREAM compose(INSTREAM inputStream);
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/Option.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/Option.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/Option.java
new file mode 100644
index 0000000..1b8935f
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/Option.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.datatorrent.api.Attribute;
+
+/**
+ * Options for the operators in the dag
+ */
+@InterfaceStability.Evolving
+public interface Option
+{
+
+  class Options
+  {
+    public static Option name(String name)
+    {
+      return new OpName(name);
+    }
+
+    public static Option prop(String name, Object value)
+    {
+      return new PropSetting(name, value);
+    }
+
+    public static <T> Option attr(Attribute<T> attr, T obj)
+    {
+      return new AttributeSetting<>(attr, obj);
+    }
+  }
+
+  /**
+   * An option used to set the name of the operator
+   */
+  class OpName implements Option
+  {
+
+    private String name;
+
+    public OpName(String name)
+    {
+      this.name = name;
+    }
+
+    public String getName()
+    {
+      return name;
+    }
+  }
+
+  /**
+   * An option used to set the property value of the operator
+   */
+  class PropSetting implements Option
+  {
+
+    private String name;
+
+    private Object val;
+
+    public PropSetting(String name, Object val)
+    {
+      this.name = name;
+      this.val = val;
+    }
+
+    public String getName()
+    {
+      return name;
+    }
+
+    public Object getVal()
+    {
+      return val;
+    }
+  }
+
+  /**
+   * An option used to set the {@link Attribute}
+   * @param <T>
+   */
+  class AttributeSetting<T> implements Option
+  {
+    private Attribute<T> attr;
+
+    private T value;
+
+    public AttributeSetting(Attribute<T> attr, T value)
+    {
+      this.attr = attr;
+      this.value = value;
+    }
+
+    public Attribute<T> getAttr()
+    {
+      return attr;
+    }
+
+    public T getValue()
+    {
+      return value;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
index 748a76a..bc99035 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
@@ -18,6 +18,22 @@
  */
 package org.apache.apex.malhar.stream.api;
 
+import java.util.List;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl;
+import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.accumulation.FoldFn;
+import org.apache.apex.malhar.stream.api.impl.accumulation.ReduceFn;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.datatorrent.lib.util.KeyValPair;
+
 /**
  * <p>
  * A stream with windowed transformation
@@ -51,6 +67,140 @@ package org.apache.apex.malhar.stream.api;
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public interface WindowedStream<T> extends ApexStream<T>
 {
+
+  /**
+   * Count of all tuples
+   * @return new stream of Integer
+   */
+  <STREAM extends WindowedStream<Tuple.WindowedTuple<Long>>> STREAM count(Option... opts);
+
+  /**
+   * Count tuples by the key<br>
+   * @param name name of the operator
+   * @param convertToKeyValue The function convert plain tuple to k,v pair
+   * @return new stream of Key Value Pair
+   */
+  <K, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, Long>>>> STREAM countByKey(Function.ToKeyValue<T, K, Long> convertToKeyValue, Option... opts);
+
+  /**
+   * Return top N tuples by the selected key
+   * @param N how many tuples you want to keep
+   * @param name name of the operator
+   * @param convertToKeyVal The function convert plain tuple to k,v pair
+   * @return new stream of Key and top N tuple of the key
+   */
+  <K, V, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, List<V>>>>> STREAM topByKey(int N, Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts);
+
+  /**
+   * Return top N tuples of all tuples in the window
+   * @param N
+   * @param name name of the operator
+   * @return new stream of topN
+   */
+  <STREAM extends WindowedStream<Tuple.WindowedTuple<List<T>>>> STREAM top(int N, Option... opts);
+
+  /**
+   * Add {@link KeyedWindowedOperatorImpl} with specified {@link Accumulation} <br>
+   * Accumulate tuples by some key within the window definition in this stream
+   * Also give a name to the accumulation
+   * @param accumulation Accumulation function you want to do
+   * @param convertToKeyVal The function convert plain tuple to k,v pair
+   * @param <K> The type of the key used to group tuples
+   * @param <V> The type of value you want to do accumulation on
+   * @param <O> The output type for each given key that you want to accumulate the value to
+   * @param <ACCU> The type of accumulation you want to keep (it can be in memory or on disk)
+   * @param <STREAM> return type
+   * @return
+   */
+  <K, V, O, ACCU, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, O>>>> STREAM accumulateByKey(Accumulation<V, ACCU, O> accumulation,
+      Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts);
+
+  /**
+   * Add {@link WindowedOperatorImpl} with specified {@link Accumulation} <br>
+   * Accumulate tuples by some key within the window definition in this stream
+   * Also give a name to the accumulation
+   * @param accumulation Accumulation function you want to do
+   * @param <O> The output type that you want to accumulate the value to
+   * @param <ACCU> The type of accumulation you want to keep (it can be in memory or on disk)
+   * @param <STREAM> return type
+   * @return
+   */
+  <O, ACCU, STREAM extends WindowedStream<Tuple.WindowedTuple<O>>> STREAM accumulate(Accumulation<T, ACCU, O> accumulation, Option... opts);
+
+  /**
+   * Add {@link WindowedOperatorImpl} with specified {@link ReduceFn} <br>
+   * Do reduce transformation<br>
+   * @param reduce reduce function
+   * @param <STREAM> return type
+   * @return new stream of same type
+   */
+  <STREAM extends WindowedStream<Tuple.WindowedTuple<T>>> STREAM reduce(ReduceFn<T> reduce, Option... opts);
+
+  /**
+   * Add {@link KeyedWindowedOperatorImpl} with specified {@link ReduceFn} <br>
+   * Reduce transformation by selected key <br>
+   * Add an operator to the DAG which merge tuple t1, t2 to new tuple by key
+   * @param reduce reduce function
+   * @param convertToKeyVal The function convert plain tuple to k,v pair
+   * @param <K> The type of key you want to group tuples by
+   * @param <V> The type of value extract from tuple T
+   * @param <STREAM> return type
+   * @return new stream of key value pair
+   */
+  <K, V, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, V>>>> STREAM reduceByKey(ReduceFn<V> reduce, Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts);
+
+
+  /**
+   * Add {@link WindowedOperatorImpl} with specified {@link FoldFn} <br>
+   * Fold transformation <br>
+   * @param fold fold function
+   * @param <O> output type of fold function
+   * @param <STREAM> return type
+   * @return
+   */
+  <O, STREAM extends WindowedStream<Tuple.WindowedTuple<O>>> STREAM fold(FoldFn<T, O> fold, Option... opts);
+
+  /**
+   * Add {@link KeyedWindowedOperatorImpl} with specified {@link FoldFn} <br>
+   * Fold transformation by key <br>
+   * @param fold fold function
+   * @param <O> Result type
+   * @return new stream of type O
+   */
+  <K, V, O, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, O>>>> STREAM foldByKey(FoldFn<V, O> fold, Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts);
+
+
+  /**
+   * Return tuples for each key for each window
+   * @param <O>
+   * @param <K>
+   * @param <STREAM>
+   * @return
+   */
+  <O, K, STREAM extends WindowedStream<KeyValPair<K, Iterable<O>>>> STREAM groupByKey(Function.ToKeyValue<T, K, O> convertToKeyVal, Option... opts);
+
+  /**
+   * Return tuples for each window
+   * @param <STREAM>
+   * @return
+   */
+  <STREAM extends WindowedStream<Iterable<T>>> STREAM group();
+
+  /**
+   * Reset the trigger settings for next transforms
+   * @param triggerOption
+   * @param <STREAM>
+   */
+  <STREAM extends WindowedStream<T>> STREAM resetTrigger(TriggerOption triggerOption);
+
+  /**
+   * Reset the allowedLateness settings for next transforms
+   * @param allowedLateness
+   * @param <STREAM>
+   */
+  <STREAM extends WindowedStream<T>> STREAM resetAllowedLateness(Duration allowedLateness);
+
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/function/Function.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/function/Function.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/function/Function.java
index f4e5e60..d516064 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/function/Function.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/function/Function.java
@@ -18,11 +18,24 @@
  */
 package org.apache.apex.malhar.stream.api.function;
 
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.stream.api.operator.FunctionOperator;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.datatorrent.lib.util.KeyValPair;
+
 /**
- * The top level function interface
+ * The top level function interface <br>
+ * The function is wrapped by {@link FunctionOperator} <br>
+ * It takes input from input port of {@link FunctionOperator} ex. {@link FunctionOperator.MapFunctionOperator#input} <br>
+ * And the output will be emitted using {@link FunctionOperator#tupleOutput} <br>
+ * Anonymous function is not fully supported. It must be <b>stateless</b> should not be defined in any static context<br>
+ * If anonymous function does not working, you can should use top level function class<br>
+ * Top level function class should have public non-arg constructor
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public interface Function
 {
   /**
@@ -45,26 +58,18 @@ public interface Function
   }
 
   /**
-   * An interface defines a reduce transformation
+   * A special map function to convert any pojo to key value pair datastructure
    * @param <T>
+   * @param <K>
+   * @param <V>
    */
-  public static interface ReduceFunction<T> extends Function
+  public static interface ToKeyValue<T, K, V> extends MapFunction<T, Tuple<KeyValPair<K, V>>>
   {
-    T reduce(T t1, T t2);
-  }
 
-  /**
-   * An interface that defines a fold transformation
-   * @param <I>
-   * @param <O>
-   */
-  public static interface FoldFunction<I, O> extends Function
-  {
-    O fold(I input, O output);
   }
 
   /**
-   * An interface that defines flatmap transforation
+   * An interface that defines flatmap transformation
    * @param <I>
    * @param <O>
    */
@@ -76,7 +81,8 @@ public interface Function
    * An interface that defines filter transformation
    * @param <T>
    */
-  public static interface FilterFunction<T> extends MapFunction<T, Boolean>
+  public static interface FilterFunction<T> extends Function
   {
+    boolean f(T input);
   }
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java
index 2ff6d51..032cb03 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexStreamImpl.java
@@ -27,28 +27,39 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.WindowOption;
+
 import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.Option;
+import org.apache.apex.malhar.stream.api.WindowedStream;
 import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.function.Function.FlatMapFunction;
 import org.apache.apex.malhar.stream.api.operator.FunctionOperator;
 import org.apache.commons.beanutils.BeanUtils;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.classification.InterfaceStability;
 
 import com.datatorrent.api.Attribute;
 import com.datatorrent.api.Context;
 import com.datatorrent.api.DAG;
 import com.datatorrent.api.LocalMode;
 import com.datatorrent.api.Operator;
-import com.datatorrent.lib.algo.UniqueCounter;
 import com.datatorrent.lib.io.ConsoleOutputOperator;
 import com.datatorrent.stram.StramLocalCluster;
 import com.datatorrent.stram.plan.logical.LogicalPlan;
 
 /**
- * Default stream implementation for ApexStream interface.
- * It creates the dag(execution plan) from stream api
+ * Default stream implementation for ApexStream interface. <br>
+ * It creates the dag(execution plan) from stream api <br>
+ * The dag won't be constructed until {@link #populateDag(DAG)} is called
  *
  * @since 3.4.0
  */
+@InterfaceStability.Evolving
 public class ApexStreamImpl<T> implements ApexStream<T>
 {
 
@@ -135,18 +146,17 @@ public class ApexStreamImpl<T> implements ApexStream<T>
     }
   }
 
+
   /**
    * Graph behind the stream
    */
-  private DagMeta graph;
-
-  private ApexStream<T> delegator;
+  protected DagMeta graph;
 
   /**
-   * Right now the stream only support single extend point
-   * You can have multiple downstream operators connect to this single extend point though
+   * Right now the stream only support single extension point
+   * You can have multiple downstream operators connect to this single extension point though
    */
-  private Brick<T> lastBrick;
+  protected Brick<T> lastBrick;
 
   public Brick<T> getLastBrick()
   {
@@ -163,13 +173,11 @@ public class ApexStreamImpl<T> implements ApexStream<T>
     graph = new DagMeta();
   }
 
-  public ApexStreamImpl(ApexStream<T> apexStream)
+  public ApexStreamImpl(ApexStreamImpl<T> apexStream)
   {
-    this.delegator = apexStream;
-    if (delegator != null && delegator instanceof ApexStreamImpl) {
-      graph = ((ApexStreamImpl)delegator).graph;
-      lastBrick = ((ApexStreamImpl<T>)delegator).lastBrick;
-    }
+    //copy the variables over to the new ApexStreamImpl
+    graph = apexStream.graph;
+    lastBrick = apexStream.lastBrick;
   }
 
   public ApexStreamImpl(DagMeta graph)
@@ -184,128 +192,52 @@ public class ApexStreamImpl<T> implements ApexStream<T>
   }
 
   @Override
-  public <O, STREAM extends ApexStream<O>> STREAM map(Function.MapFunction<T, O> mf)
-  {
-    return map(mf.toString(), mf);
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public <O, STREAM extends ApexStream<O>> STREAM map(String name, Function.MapFunction<T, O> mf)
+  public <O, STREAM extends ApexStream<O>> STREAM map(Function.MapFunction<T, O> mf, Option... opts)
   {
     FunctionOperator.MapFunctionOperator<T, O> opt = new FunctionOperator.MapFunctionOperator<>(mf);
-    return (STREAM)addOperator(name, opt, opt.input, opt.output);
+    return addOperator(opt, opt.input, opt.output, opts);
   }
 
-  @Override
-  public <O, STREAM extends ApexStream<O>> STREAM flatMap(Function.FlatMapFunction<T, O> flatten)
-  {
-    return flatMap(flatten.toString(), flatten);
-  }
 
   @Override
-  @SuppressWarnings("unchecked")
-  public <O, STREAM extends ApexStream<O>> STREAM flatMap(String name, Function.FlatMapFunction<T, O> flatten)
+  public <O, STREAM extends ApexStream<O>> STREAM flatMap(FlatMapFunction<T, O> flatten, Option... opts)
   {
     FunctionOperator.FlatMapFunctionOperator<T, O> opt = new FunctionOperator.FlatMapFunctionOperator<>(flatten);
-    return (STREAM)addOperator(name, opt, opt.input, opt.output);
-  }
-
-  @Override
-  public <STREAM extends ApexStream<T>> STREAM filter(final Function.FilterFunction<T> filter)
-  {
-    return filter(filter.toString(), filter);
+    return addOperator(opt, opt.input, opt.output, opts);
   }
 
   @Override
   @SuppressWarnings("unchecked")
-  public <STREAM extends ApexStream<T>> STREAM filter(String name, final Function.FilterFunction<T> filter)
+  public <STREAM extends ApexStream<T>> STREAM filter(final Function.FilterFunction<T> filter, Option... opts)
   {
     FunctionOperator.FilterFunctionOperator<T> filterFunctionOperator = new FunctionOperator.FilterFunctionOperator<>(filter);
-    return (STREAM)addOperator(name, filterFunctionOperator, filterFunctionOperator.input, filterFunctionOperator.output);
-  }
-
-  @Override
-  public <STREAM extends ApexStream<T>> STREAM reduce(Function.ReduceFunction<T> reduce)
-  {
-    return reduce(reduce.toString(), reduce);
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public <STREAM extends ApexStream<T>> STREAM reduce(String name, Function.ReduceFunction<T> reduce)
-  {
-    FunctionOperator.ReduceFunctionOperator<T> opt = new FunctionOperator.ReduceFunctionOperator<>(reduce);
-    return (STREAM)addOperator(name, opt, opt.input, opt.output);
-  }
-
-  @Override
-  public <O, STREAM extends ApexStream<O>> STREAM fold(final O initialValue, Function.FoldFunction<T, O> fold)
-  {
-    return fold(fold.toString(), initialValue, fold);
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public <O, STREAM extends ApexStream<O>> STREAM fold(String name, O initialValue, Function.FoldFunction<T, O> fold)
-  {
-    FunctionOperator.FoldFunctionOperator<T, O> opt = new FunctionOperator.FoldFunctionOperator<>(fold, initialValue);
-    return (STREAM)addOperator(name, opt, opt.input, opt.output);
-  }
-
-  @Override
-  public <STREAM extends ApexStream<Integer>> STREAM count()
-  {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public <STREAM extends ApexStream<Map<Object, Integer>>> STREAM countByKey(int key)
-  {
-    throw new UnsupportedOperationException();
+    return addOperator(filterFunctionOperator, filterFunctionOperator.input, filterFunctionOperator.output, opts);
   }
 
-  @Override
-  @SuppressWarnings("unchecked")
-  public <STREAM extends ApexStream<Map<Object, Integer>>> STREAM countByKey()
+  public <STREAM extends ApexStream<Map.Entry<Object, Integer>>> STREAM countByElement()
   {
-    // Needs to change the unique counter to support keys
-    UniqueCounter<Object> uniqueCounter = new UniqueCounter<>();
-    uniqueCounter.setCumulative(true);
-    Operator.OutputPort<? extends Map<Object, Integer>> resultPort = uniqueCounter.count;
-    return (STREAM)addOperator("CounterByKey", uniqueCounter, (Operator.InputPort<T>)uniqueCounter.data, resultPort);
+    return null;
   }
 
   @Override
-  public <O, STREAM extends ApexStream<O>> STREAM addOperator(Operator op, Operator.InputPort<T> inputPort, Operator.OutputPort<O> outputPort)
+  public <O, STREAM extends ApexStream<O>> STREAM endWith(Operator op, Operator.InputPort<T> inputPort, Option... opts)
   {
-    return addOperator(op.toString(), op, inputPort, outputPort);
+    return (STREAM)addOperator(op, inputPort, null, opts);
   }
 
-
   @Override
   @SuppressWarnings("unchecked")
-  public <O, STREAM extends ApexStream<O>> STREAM addOperator(String opName, Operator op, Operator.InputPort<T> inputPort, Operator.OutputPort<O> outputPort)
+  public  <O, STREAM extends ApexStream<O>> STREAM addOperator(Operator op, Operator.InputPort<T> inputPort, Operator.OutputPort<O> outputPort, Option... opts)
   {
 
-    if (delegator != null) {
-      ApexStreamImpl<O> apexStream = delegator.addOperator(opName, op, inputPort, outputPort);
-      try {
-        return (STREAM)this.getClass().getConstructor(ApexStream.class).newInstance(apexStream);
-      } catch (Exception e) {
-        throw new RuntimeException("You have to override the default constructor with ApexStreamImpl as delegator");
-      }
-    }
-
     checkArguments(op, inputPort, outputPort);
 
     DagMeta.NodeMeta nm = null;
 
     if (lastBrick == null) {
-      nm = graph.addNode(opName, op, null, null, inputPort);
+      nm = graph.addNode(op, null, null, inputPort, opts);
     } else {
-
-      nm = graph.addNode(opName, op, lastBrick.nodeMeta, lastBrick.lastOutput, inputPort);
+      nm = graph.addNode(op, lastBrick.nodeMeta, lastBrick.lastOutput, inputPort, opts);
     }
 
     Brick<O> newBrick = new Brick<>();
@@ -315,9 +247,25 @@ public class ApexStreamImpl<T> implements ApexStream<T>
       newBrick.lastStream = Pair.<Operator.OutputPort, Operator.InputPort>of(lastBrick.lastOutput, inputPort);
     }
 
-    return (STREAM)new ApexStreamImpl<>(this.graph, newBrick);
+    if (this.getClass() == ApexStreamImpl.class || this.getClass() == ApexWindowedStreamImpl.class) {
+      return (STREAM)newStream(this.graph, newBrick);
+    } else {
+      try {
+        return (STREAM)this.getClass().getConstructor(ApexStreamImpl.class).newInstance(newStream(this.graph, newBrick));
+      } catch (Exception e) {
+        throw new RuntimeException("You have to override the default constructor with ApexStreamImpl as default parameter", e);
+      }
+    }
+
+  }
+
+  @Override
+  public <O, INSTREAM extends ApexStream<T>, OUTSTREAM extends ApexStream<O>> OUTSTREAM addCompositeStreams(CompositeStreamTransform<INSTREAM, OUTSTREAM> compositeStreamTransform)
+  {
+    return compositeStreamTransform.compose((INSTREAM)this);
   }
 
+
   /* Check to see if inputPort and outputPort belongs to the operator */
   private void checkArguments(Operator op, Operator.InputPort inputPort, Operator.OutputPort outputPort)
   {
@@ -362,8 +310,8 @@ public class ApexStreamImpl<T> implements ApexStream<T>
   public ApexStreamImpl<T> print()
   {
     ConsoleOutputOperator consoleOutputOperator = new ConsoleOutputOperator();
-    addOperator(IDGenerator.generateOperatorIDWithUUID(consoleOutputOperator.getClass()), consoleOutputOperator,
-        (Operator.InputPort<T>)consoleOutputOperator.input, null);
+    addOperator(consoleOutputOperator,
+        (Operator.InputPort<T>)consoleOutputOperator.input, null, Option.Options.name(IDGenerator.generateOperatorIDWithUUID(consoleOutputOperator.getClass())));
     return this;
   }
 
@@ -469,6 +417,7 @@ public class ApexStreamImpl<T> implements ApexStream<T>
   {
     LocalMode lma = LocalMode.newInstance();
     populateDag(lma.getDAG());
+    DAG dag = lma.getDAG();
     LocalMode.Controller lc = lma.getController();
     if (lc instanceof StramLocalCluster) {
       ((StramLocalCluster)lc).setExitCondition(exitCondition);
@@ -493,5 +442,36 @@ public class ApexStreamImpl<T> implements ApexStream<T>
     //TODO need an api to submit the StreamingApplication to cluster
   }
 
+  @Override
+  public WindowedStream<T> window(WindowOption option)
+  {
+    return window(option, null, null);
+  }
+
+  @Override
+  public WindowedStream<T> window(WindowOption windowOption, TriggerOption triggerOption)
+  {
+    return window(windowOption, triggerOption, null);
+  }
+
+  @Override
+  public WindowedStream<T> window(WindowOption windowOption, TriggerOption triggerOption, Duration allowLateness)
+  {
+    ApexWindowedStreamImpl<T> windowedStream = new ApexWindowedStreamImpl<>();
+    windowedStream.lastBrick = lastBrick;
+    windowedStream.graph = graph;
+    windowedStream.windowOption = windowOption;
+    windowedStream.triggerOption = triggerOption;
+    windowedStream.allowedLateness = allowLateness;
+    return windowedStream;
+  }
+
+  protected <O> ApexStream<O> newStream(DagMeta graph, Brick<O> newBrick)
+  {
+    ApexStreamImpl<O> newstream = new ApexStreamImpl<>();
+    newstream.graph = graph;
+    newstream.lastBrick = newBrick;
+    return newstream;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/266b0411/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java
new file mode 100644
index 0000000..a293ea8
--- /dev/null
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java
@@ -0,0 +1,275 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.api.impl;
+
+import java.util.List;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.lib.window.WindowState;
+
+import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedKeyedStorage;
+import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedStorage;
+import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl;
+import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.Option;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+
+import org.apache.apex.malhar.stream.api.impl.accumulation.Count;
+import org.apache.apex.malhar.stream.api.impl.accumulation.FoldFn;
+import org.apache.apex.malhar.stream.api.impl.accumulation.ReduceFn;
+import org.apache.apex.malhar.stream.api.impl.accumulation.TopN;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.datatorrent.lib.util.KeyValPair;
+
+/**
+ * Default windowed stream implementation for WindowedStream interface.
+ * It adds more windowed transform for Stream interface
+ *
+ * @since 3.4.0
+ */
+@InterfaceStability.Evolving
+public class ApexWindowedStreamImpl<T> extends ApexStreamImpl<T> implements WindowedStream<T>
+{
+
+  protected WindowOption windowOption;
+
+  protected TriggerOption triggerOption;
+
+  protected Duration allowedLateness;
+
+  private class ConvertFn<T> implements Function.MapFunction<T, Tuple<T>>
+  {
+
+    @Override
+    public Tuple<T> f(T input)
+    {
+      if (input instanceof Tuple.TimestampedTuple) {
+        return (Tuple.TimestampedTuple)input;
+      } else {
+        return new Tuple.TimestampedTuple<>(System.currentTimeMillis(), input);
+      }
+    }
+  }
+
+
+  public ApexWindowedStreamImpl()
+  {
+  }
+
+  @Override
+  public <STREAM extends WindowedStream<Tuple.WindowedTuple<Long>>> STREAM count(Option... opts)
+  {
+    Function.MapFunction<T, Tuple<Long>> kVMap = new Function.MapFunction<T, Tuple<Long>>()
+    {
+      @Override
+      public Tuple<Long> f(T input)
+      {
+        if (input instanceof Tuple.TimestampedTuple) {
+          return new Tuple.TimestampedTuple<>(((Tuple.TimestampedTuple)input).getTimestamp(), 1L);
+        } else {
+          return new Tuple.TimestampedTuple<>(System.currentTimeMillis(), 1L);
+        }
+      }
+    };
+
+    WindowedStream<Tuple<Long>> innerstream = map(kVMap);
+    WindowedOperatorImpl<Long, MutableLong, Long> windowedOperator = createWindowedOperator(new Count());
+    return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts);
+  }
+
+  @Override
+  public <K, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, Long>>>> STREAM countByKey(Function.ToKeyValue<T, K, Long> convertToKeyValue, Option... opts)
+  {
+    WindowedStream<Tuple<KeyValPair<K, Long>>> kvstream = map(convertToKeyValue);
+    KeyedWindowedOperatorImpl<K, Long, MutableLong, Long> keyedWindowedOperator = createKeyedWindowedOperator(new Count());
+    return kvstream.addOperator(keyedWindowedOperator, keyedWindowedOperator.input, keyedWindowedOperator.output, opts);
+  }
+
+  @Override
+  public <K, V, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, List<V>>>>> STREAM topByKey(int N, Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts)
+  {
+    TopN<V> top = new TopN<>();
+    top.setN(N);
+    WindowedStream<Tuple<KeyValPair<K, V>>> kvstream = map(convertToKeyVal);
+    KeyedWindowedOperatorImpl<K, V, List<V>, List<V>> keyedWindowedOperator = createKeyedWindowedOperator(top);
+    return kvstream.addOperator(keyedWindowedOperator, keyedWindowedOperator.input, keyedWindowedOperator.output, opts);
+  }
+
+  @Override
+  public <STREAM extends WindowedStream<Tuple.WindowedTuple<List<T>>>> STREAM top(int N, Option... opts)
+  {
+
+    TopN<T> top = new TopN<>();
+    top.setN(N);
+    WindowedStream<Tuple<T>> innerstream = map(new ConvertFn<T>());
+    WindowedOperatorImpl<T, List<T>, List<T>> windowedOperator = createWindowedOperator(top);
+    return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts);
+  }
+
+
+  @Override
+  public <K, V, O, ACCU, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, O>>>> STREAM accumulateByKey(Accumulation<V, ACCU, O> accumulation,
+      Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts)
+  {
+    WindowedStream<Tuple<KeyValPair<K, V>>> kvstream = map(convertToKeyVal);
+    KeyedWindowedOperatorImpl<K, V, ACCU, O> keyedWindowedOperator = createKeyedWindowedOperator(accumulation);
+    return kvstream.addOperator(keyedWindowedOperator, keyedWindowedOperator.input, keyedWindowedOperator.output, opts);
+  }
+
+
+  @Override
+  public <O, ACCU, STREAM extends WindowedStream<Tuple.WindowedTuple<O>>> STREAM accumulate(Accumulation<T, ACCU, O> accumulation, Option... opts)
+  {
+    WindowedStream<Tuple<T>> innerstream = map(new ConvertFn<T>());
+    WindowedOperatorImpl<T, ACCU, O> windowedOperator = createWindowedOperator(accumulation);
+    return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts);
+  }
+
+
+  @Override
+  public <STREAM extends WindowedStream<Tuple.WindowedTuple<T>>> STREAM reduce(ReduceFn<T> reduce, Option... opts)
+  {
+    WindowedStream<Tuple<T>> innerstream = map(new ConvertFn<T>());
+    WindowedOperatorImpl<T, T, T> windowedOperator = createWindowedOperator(reduce);
+    return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts);
+  }
+
+  @Override
+  public <K, V, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, V>>>> STREAM reduceByKey(ReduceFn<V> reduce, Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts)
+  {
+    WindowedStream<Tuple<KeyValPair<K, V>>> kvstream = map(convertToKeyVal);
+    KeyedWindowedOperatorImpl<K, V, V, V> keyedWindowedOperator = createKeyedWindowedOperator(reduce);
+    return kvstream.addOperator(keyedWindowedOperator, keyedWindowedOperator.input, keyedWindowedOperator.output, opts);
+  }
+
+  @Override
+  public <O, STREAM extends WindowedStream<Tuple.WindowedTuple<O>>> STREAM fold(FoldFn<T, O> fold, Option... opts)
+  {
+    WindowedStream<Tuple<T>> innerstream = map(new ConvertFn<T>());
+    WindowedOperatorImpl<T, O, O> windowedOperator = createWindowedOperator(fold);
+    return innerstream.addOperator(windowedOperator, windowedOperator.input, windowedOperator.output, opts);
+  }
+
+  @Override
+  public <K, V, O, STREAM extends WindowedStream<Tuple.WindowedTuple<KeyValPair<K, O>>>> STREAM foldByKey(FoldFn<V, O> fold, Function.ToKeyValue<T, K, V> convertToKeyVal, Option... opts)
+  {
+    WindowedStream<Tuple<KeyValPair<K, V>>> kvstream = map(convertToKeyVal);
+    KeyedWindowedOperatorImpl<K, V, O, O> keyedWindowedOperator = createKeyedWindowedOperator(fold);
+    return kvstream.addOperator(keyedWindowedOperator, keyedWindowedOperator.input, keyedWindowedOperator.output, opts);
+
+  }
+
+  @Override
+  public <O, K, STREAM extends WindowedStream<KeyValPair<K, Iterable<O>>>> STREAM groupByKey(Function.ToKeyValue<T, K, O> convertToKeyVal, Option... opts)
+  {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <STREAM extends WindowedStream<Iterable<T>>> STREAM group()
+  {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <STREAM extends WindowedStream<T>> STREAM resetTrigger(TriggerOption option)
+  {
+    triggerOption = option;
+    return (STREAM)this;
+  }
+
+  @Override
+  public <STREAM extends WindowedStream<T>> STREAM resetAllowedLateness(Duration allowedLateness)
+  {
+    this.allowedLateness = allowedLateness;
+    return (STREAM)this;
+  }
+
+  @Override
+  protected <O> ApexStream<O> newStream(DagMeta graph, Brick<O> newBrick)
+  {
+    ApexWindowedStreamImpl<O> newstream = new ApexWindowedStreamImpl<>();
+    newstream.graph = graph;
+    newstream.lastBrick = newBrick;
+    newstream.windowOption = this.windowOption;
+    newstream.triggerOption = this.triggerOption;
+    newstream.allowedLateness = this.allowedLateness;
+    return newstream;
+  }
+
+  /**
+   * Create the windowed operator for windowed transformation
+   * @param accumulationFn
+   * @param <IN>
+   * @param <ACCU>
+   * @param <OUT>
+   * @return
+   */
+  private <IN, ACCU, OUT> WindowedOperatorImpl<IN, ACCU, OUT> createWindowedOperator(Accumulation<IN, ACCU, OUT> accumulationFn)
+  {
+    WindowedOperatorImpl<IN, ACCU, OUT> windowedOperator = new WindowedOperatorImpl<>();
+    //TODO use other default setting in the future
+    windowedOperator.setDataStorage(new InMemoryWindowedStorage<ACCU>());
+    windowedOperator.setRetractionStorage(new InMemoryWindowedStorage<OUT>());
+    windowedOperator.setWindowStateStorage(new InMemoryWindowedStorage<WindowState>());
+    if (windowOption != null) {
+      windowedOperator.setWindowOption(windowOption);
+    }
+    if (triggerOption != null) {
+      windowedOperator.setTriggerOption(triggerOption);
+    }
+    if (allowedLateness != null) {
+      windowedOperator.setAllowedLateness(allowedLateness);
+    }
+    windowedOperator.setAccumulation(accumulationFn);
+    return windowedOperator;
+  }
+
+  private <K, V, ACCU, OUT> KeyedWindowedOperatorImpl<K, V, ACCU, OUT> createKeyedWindowedOperator(Accumulation<V, ACCU, OUT> accumulationFn)
+  {
+    KeyedWindowedOperatorImpl<K, V, ACCU, OUT> keyedWindowedOperator = new KeyedWindowedOperatorImpl<>();
+
+    //TODO use other default setting in the future
+    keyedWindowedOperator.setDataStorage(new InMemoryWindowedKeyedStorage<K, ACCU>());
+    keyedWindowedOperator.setRetractionStorage(new InMemoryWindowedKeyedStorage<K, OUT>());
+    keyedWindowedOperator.setWindowStateStorage(new InMemoryWindowedStorage<WindowState>());
+    if (windowOption != null) {
+      keyedWindowedOperator.setWindowOption(windowOption);
+    }
+    if (triggerOption != null) {
+      keyedWindowedOperator.setTriggerOption(triggerOption);
+    }
+    if (allowedLateness != null) {
+      keyedWindowedOperator.setAllowedLateness(allowedLateness);
+    }
+
+    keyedWindowedOperator.setAccumulation(accumulationFn);
+    return keyedWindowedOperator;
+  }
+
+}


[4/6] apex-malhar git commit: Added Beam Examples and Implementations of Accumulation.

Posted by th...@apache.org.
Added Beam Examples and Implementations of Accumulation.


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

Branch: refs/heads/master
Commit: dcca7752a8ee966d67602a1b7cb8fbacdb8ed59d
Parents: 266b041
Author: Shunxin <lu...@hotmail.com>
Authored: Wed Aug 24 13:12:20 2016 -0700
Committer: Siyuan Hua <hs...@apache.org>
Committed: Thu Aug 25 09:26:03 2016 -0700

----------------------------------------------------------------------
 demos/highlevelapi/pom.xml                      |  98 +++-
 .../malhar/stream/sample/MinimalWordCount.java  | 126 +++++
 .../malhar/stream/sample/WindowedWordCount.java | 277 ++++++++++
 .../stream/sample/complete/AutoComplete.java    | 308 +++++++++++
 .../sample/complete/CompletionCandidate.java    |  87 ++++
 .../stream/sample/complete/PojoEvent.java       |  44 ++
 .../sample/complete/StreamingWordExtract.java   | 160 ++++++
 .../stream/sample/complete/TopNByKey.java       | 118 +++++
 .../sample/complete/TopWikipediaSessions.java   | 340 ++++++++++++
 .../stream/sample/complete/TrafficRoutes.java   | 521 +++++++++++++++++++
 .../sample/complete/TwitterAutoComplete.java    | 251 +++++++++
 .../sample/cookbook/CombinePerKeyExamples.java  | 212 ++++----
 .../stream/sample/cookbook/DeDupExample.java    | 124 +++++
 .../stream/sample/cookbook/InputPojo.java       |  76 +++
 .../sample/cookbook/MaxPerKeyExamples.java      | 203 ++++++++
 .../stream/sample/cookbook/OutputPojo.java      |  54 ++
 .../stream/sample/cookbook/TriggerExample.java  | 137 +++--
 .../src/main/resources/META-INF/properties.xml  | 141 +++++
 .../stream/sample/MinimalWordCountTest.java     |  61 +++
 .../stream/sample/WindowedWordCountTest.java    |  90 ++++
 .../sample/complete/AutoCompleteTest.java       |  66 +++
 .../complete/StreamingWordExtractTest.java      | 144 +++++
 .../complete/TopWikipediaSessionsTest.java      |  73 +++
 .../sample/complete/TrafficRoutesTest.java      |  66 +++
 .../complete/TwitterAutoCompleteTest.java       |  66 +++
 .../cookbook/CombinePerKeyExamplesTest.java     |  56 ++
 .../sample/cookbook/DeDupExampleTest.java       |  59 +++
 .../sample/cookbook/MaxPerKeyExamplesTest.java  | 210 ++++++++
 .../src/test/resources/data/word.txt            |   2 +
 .../src/test/resources/log4j.properties         |  45 ++
 .../src/test/resources/sampletweets.txt         | 207 ++++++++
 .../src/test/resources/wordcount/word.txt       |   8 +
 demos/pom.xml                                   |  13 +-
 .../lib/window/impl/accumulation/Average.java   |  64 +++
 .../lib/window/impl/accumulation/Count.java     |  61 +++
 .../lib/window/impl/accumulation/FoldFn.java    |  65 +++
 .../lib/window/impl/accumulation/Group.java     |  63 +++
 .../lib/window/impl/accumulation/Max.java       |  75 +++
 .../lib/window/impl/accumulation/Min.java       |  76 +++
 .../lib/window/impl/accumulation/ReduceFn.java  |  65 +++
 .../impl/accumulation/RemoveDuplicates.java     |  72 +++
 .../lib/window/impl/accumulation/SumDouble.java |  60 +++
 .../lib/window/impl/accumulation/SumFloat.java  |  60 +++
 .../lib/window/impl/accumulation/SumInt.java    |  60 +++
 .../lib/window/impl/accumulation/SumLong.java   |  60 +++
 .../lib/window/impl/accumulation/TopN.java      | 106 ++++
 .../lib/window/impl/accumulation/TopNByKey.java | 114 ++++
 .../window/impl/accumulation/AverageTest.java   |  41 ++
 .../window/impl/accumulation/FoldFnTest.java    | 129 +++++
 .../lib/window/impl/accumulation/GroupTest.java |  42 ++
 .../lib/window/impl/accumulation/MaxTest.java   |  53 ++
 .../lib/window/impl/accumulation/MinTest.java   |  53 ++
 .../window/impl/accumulation/ReduceFnTest.java  |  50 ++
 .../impl/accumulation/RemoveDuplicatesTest.java |  42 ++
 .../lib/window/impl/accumulation/SumTest.java   |  57 ++
 .../window/impl/accumulation/TopNByKeyTest.java |  75 +++
 .../apex/malhar/stream/api/WindowedStream.java  |   4 +-
 .../stream/api/impl/ApexWindowedStreamImpl.java |  11 +-
 .../stream/api/impl/accumulation/Count.java     |  61 ---
 .../stream/api/impl/accumulation/FoldFn.java    |  65 ---
 .../stream/api/impl/accumulation/ReduceFn.java  |  65 ---
 .../stream/api/impl/accumulation/TopN.java      | 107 ----
 stream/src/test/resources/words/word.txt        |   2 +
 63 files changed, 5820 insertions(+), 481 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/pom.xml
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/pom.xml b/demos/highlevelapi/pom.xml
index c669681..cde0c83 100644
--- a/demos/highlevelapi/pom.xml
+++ b/demos/highlevelapi/pom.xml
@@ -34,21 +34,107 @@
     <version>3.5.0-SNAPSHOT</version>
   </parent>
 
-  <properties>
-    <skipTests>true</skipTests>
-  </properties>
+  <build>
+    <plugins>
+
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>1.9.1</version>
+        <executions>
+          <execution>
+            <id>attach-artifacts</id>
+            <phase>package</phase>
+            <goals>
+              <goal>attach-artifact</goal>
+            </goals>
+            <configuration>
+              <artifacts>
+                <artifact>
+                  <file>target/${project.artifactId}-${project.version}.apa</file>
+                  <type>apa</type>
+                </artifact>
+              </artifacts>
+              <skipAttach>false</skipAttach>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+    </plugins>
+  </build>
 
   <dependencies>
     <dependency>
-      <groupId>cglib</groupId>
-      <artifactId>cglib</artifactId>
-      <version>3.2.1</version>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>4.10</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <!-- required by twitter demo -->
+      <groupId>org.twitter4j</groupId>
+      <artifactId>twitter4j-core</artifactId>
+      <version>4.0.4</version>
+    </dependency>
+    <dependency>
+      <!-- required by twitter demo -->
+      <groupId>org.twitter4j</groupId>
+      <artifactId>twitter4j-stream</artifactId>
+      <version>4.0.4</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.apex</groupId>
+      <artifactId>malhar-contrib</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.apex</groupId>
+      <artifactId>malhar-stream</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>*</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.hsqldb</groupId>
+      <artifactId>hsqldb</artifactId>
+      <version>2.3.1</version>
+    </dependency>
+    <dependency>
+      <groupId>com.h2database</groupId>
+      <artifactId>h2</artifactId>
+      <version>1.4.192</version>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>joda-time</groupId>
       <artifactId>joda-time</artifactId>
       <version>2.9.1</version>
     </dependency>
+    <dependency>
+      <!--This dependency is needed for StreamingWordExtractTest-->
+      <groupId>org.codehaus.janino</groupId>
+      <artifactId>commons-compiler</artifactId>
+      <version>2.7.8</version>
+      <type>jar</type>
+    </dependency>
+    <dependency>
+      <!--This dependency is needed for StreamingWordExtractTest-->
+      <groupId>org.codehaus.janino</groupId>
+      <artifactId>janino</artifactId>
+      <version>2.7.8</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/MinimalWordCount.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/MinimalWordCount.java
new file mode 100644
index 0000000..671cc72
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/MinimalWordCount.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.lib.util.KeyValPair;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * Beam MinimalWordCount Example
+ */
+@ApplicationAnnotation(name = "MinimalWordCount")
+public class MinimalWordCount implements StreamingApplication
+{
+  public static class Collector extends BaseOperator
+  {
+    static Map<String, Long> result;
+    private static boolean done = false;
+  
+    public static boolean isDone()
+    {
+      return done;
+    }
+  
+    @Override
+    public void setup(Context.OperatorContext context)
+    {
+      done = false;
+      result = new HashMap<>();
+    }
+    
+    public final transient DefaultInputPort<KeyValPair<String, Long>> input = new DefaultInputPort<KeyValPair<String, Long>>()
+    {
+      @Override
+      public void process(KeyValPair<String, Long> tuple)
+      {
+        if (tuple.getKey().equals("bye")) {
+          done = true;
+        }
+        result.put(tuple.getKey(), tuple.getValue());
+      }
+    };
+  }
+  
+  /**
+   * Populate the dag using High-Level API.
+   * @param dag
+   * @param conf
+   */
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    Collector collector = new Collector();
+    // Create a stream reading from a file line by line using StreamFactory.
+    StreamFactory.fromFolder("./src/test/resources/wordcount", name("textInput"))
+        // Use a flatmap transformation to extract words from the incoming stream of lines.
+        .flatMap(new Function.FlatMapFunction<String, String>()
+        {
+          @Override
+          public Iterable<String> f(String input)
+          {
+            return Arrays.asList(input.split("[^a-zA-Z']+"));
+          
+          }
+        }, name("ExtractWords"))
+        // Apply windowing to the stream for counting, in this case, the window option is global window.
+        .window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
+        // Count the appearances of every word.
+        .countByKey(new Function.ToKeyValue<String, String, Long>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, Long>> f(String input)
+          {
+            return new Tuple.PlainTuple<KeyValPair<String, Long>>(new KeyValPair<String, Long>(input, 1L));
+          }
+        }, name("countByKey"))
+        // Format the counting result to a readable format by unwrapping the tuples.
+        .map(new Function.MapFunction<Tuple.WindowedTuple<KeyValPair<String, Long>>, KeyValPair<String, Long>>()
+        {
+          @Override
+          public KeyValPair<String, Long> f(Tuple.WindowedTuple<KeyValPair<String, Long>> input)
+          {
+            return input.getValue();
+          }
+        }, name("FormatResults"))
+        // Print the result.
+        .print()
+        // Attach a collector to the stream to collect results.
+        .endWith(collector, collector.input, name("Collector"))
+        // populate the dag using the stream.
+        .populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/WindowedWordCount.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/WindowedWordCount.java
new file mode 100644
index 0000000..6a6777e
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/WindowedWordCount.java
@@ -0,0 +1,277 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.joda.time.Duration;
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Throwables;
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.lib.util.KeyValPair;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * Beam WindowedWordCount Example.
+ */
+@ApplicationAnnotation(name = "WindowedWordCount")
+public class WindowedWordCount implements StreamingApplication
+{
+  static final int WINDOW_SIZE = 1;  // Default window duration in minutes
+  
+  /**
+   * A input operator that reads from and output a file line by line to downstream with a time gap between
+   * every two lines.
+   */
+  public static class TextInput extends BaseOperator implements InputOperator
+  {
+    private static boolean done = false;
+    
+    public final transient DefaultOutputPort<String> output = new DefaultOutputPort<>();
+    
+    private transient BufferedReader reader;
+  
+    public static boolean isDone()
+    {
+      return done;
+    }
+  
+    @Override
+    public void setup(Context.OperatorContext context)
+    {
+      done = false;
+      initReader();
+    }
+    
+    private void initReader()
+    {
+      try {
+        InputStream resourceStream = this.getClass().getResourceAsStream("/wordcount/word.txt");
+        reader = new BufferedReader(new InputStreamReader(resourceStream));
+      } catch (Exception ex) {
+        throw Throwables.propagate(ex);
+      }
+    }
+    
+    @Override
+    public void teardown()
+    {
+      IOUtils.closeQuietly(reader);
+    }
+    
+    @Override
+    public void emitTuples()
+    {
+      try {
+        String line = reader.readLine();
+        if (line == null) {
+          done = true;
+          reader.close();
+          Thread.sleep(1000);
+        } else {
+          this.output.emit(line);
+        }
+        Thread.sleep(50);
+      } catch (IOException ex) {
+        throw new RuntimeException(ex);
+      } catch (InterruptedException e) {
+        throw Throwables.propagate(e);
+      }
+    }
+  }
+  
+  public static class Collector extends BaseOperator
+  {
+    private static Map<KeyValPair<Long, String>, Long> result = new HashMap<>();
+  
+    public static Map<KeyValPair<Long, String>, Long> getResult()
+    {
+      return result;
+    }
+  
+    public final transient DefaultInputPort<PojoEvent> input = new DefaultInputPort<PojoEvent>()
+    {
+      @Override
+      public void process(PojoEvent tuple)
+      {
+        result.put(new KeyValPair<Long, String>(tuple.getTimestamp(), tuple.getWord()), tuple.getCount());
+      }
+    };
+  }
+  
+  /**
+   * A Pojo Tuple class used for outputting result to JDBC.
+   */
+  public static class PojoEvent
+  {
+    private String word;
+    private long count;
+    private long timestamp;
+  
+    @Override
+    public String toString()
+    {
+      return "PojoEvent (word=" + getWord() + ", count=" + getCount() + ", timestamp=" + getTimestamp() + ")";
+    }
+  
+    public String getWord()
+    {
+      return word;
+    }
+  
+    public void setWord(String word)
+    {
+      this.word = word;
+    }
+  
+    public long getCount()
+    {
+      return count;
+    }
+  
+    public void setCount(long count)
+    {
+      this.count = count;
+    }
+  
+    public long getTimestamp()
+    {
+      return timestamp;
+    }
+  
+    public void setTimestamp(long timestamp)
+    {
+      this.timestamp = timestamp;
+    }
+  }
+  
+  /**
+   * A map function that wrap the input string with a random generated timestamp.
+   */
+  public static class AddTimestampFn implements Function.MapFunction<String, Tuple.TimestampedTuple<String>>
+  {
+    private static final Duration RAND_RANGE = Duration.standardMinutes(10);
+    private final Long minTimestamp;
+    
+    AddTimestampFn()
+    {
+      this.minTimestamp = System.currentTimeMillis();
+    }
+    
+    @Override
+    public Tuple.TimestampedTuple<String> f(String input)
+    {
+      // Generate a timestamp that falls somewhere in the past two hours.
+      long randMillis = (long)(Math.random() * RAND_RANGE.getMillis());
+      long randomTimestamp = minTimestamp + randMillis;
+
+      return new Tuple.TimestampedTuple<>(randomTimestamp, input);
+    }
+  }
+  
+  /** A MapFunction that converts a Word and Count into a PojoEvent. */
+  public static class FormatAsTableRowFn implements Function.MapFunction<Tuple.WindowedTuple<KeyValPair<String, Long>>, PojoEvent>
+  {
+    @Override
+    public PojoEvent f(Tuple.WindowedTuple<KeyValPair<String, Long>> input)
+    {
+      PojoEvent row = new PojoEvent();
+      row.setTimestamp(input.getTimestamp());
+      row.setCount(input.getValue().getValue());
+      row.setWord(input.getValue().getKey());
+      return row;
+    }
+  }
+  
+  /**
+   * Populate dag with High-Level API.
+   * @param dag
+   * @param conf
+   */
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    TextInput input = new TextInput();
+    Collector collector = new Collector();
+    
+    // Create stream from the TextInput operator.
+    ApexStream<Tuple.TimestampedTuple<String>> stream = StreamFactory.fromInput(input, input.output, name("input"))
+      
+        // Extract all the words from the input line of text.
+        .flatMap(new Function.FlatMapFunction<String, String>()
+        {
+          @Override
+          public Iterable<String> f(String input)
+          {
+            return Arrays.asList(input.split("[\\p{Punct}\\s]+"));
+          }
+        }, name("ExtractWords"))
+      
+        // Wrap the word with a randomly generated timestamp.
+        .map(new AddTimestampFn(), name("AddTimestampFn"));
+    
+   
+    // apply window and trigger option.
+    // TODO: change trigger option to atWaterMark when available.
+    WindowedStream<Tuple.TimestampedTuple<String>> windowedWords = stream
+        .window(new WindowOption.TimeWindows(Duration.standardMinutes(WINDOW_SIZE)),
+        new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1));
+    
+    
+    WindowedStream<PojoEvent> wordCounts =
+        // Perform a countByKey transformation to count the appearance of each word in every time window.
+        windowedWords.countByKey(new Function.ToKeyValue<Tuple.TimestampedTuple<String>, String, Long>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, Long>> f(Tuple.TimestampedTuple<String> input)
+          {
+            return new Tuple.TimestampedTuple<KeyValPair<String, Long>>(input.getTimestamp(),
+              new KeyValPair<String, Long>(input.getValue(), 1L));
+          }
+        }, name("count words"))
+          
+        // Format the output and print out the result.
+        .map(new FormatAsTableRowFn(), name("FormatAsTableRowFn")).print();
+    
+    wordCounts.endWith(collector, collector.input, name("Collector")).populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/AutoComplete.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/AutoComplete.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/AutoComplete.java
new file mode 100644
index 0000000..29c8cf9
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/AutoComplete.java
@@ -0,0 +1,308 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.Window;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Throwables;
+
+import com.datatorrent.api.Context.OperatorContext;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.lib.util.KeyValPair;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * An example that computes the most popular hash tags
+ * for every prefix, which can be used for auto-completion.
+ * This application is identical to TwitterAutoComplete, except it's
+ * reading from a file. This application is mainly for local testing
+ * purpose.
+ *
+ * <p>This will update the datastore every 10 seconds based on the last
+ * 30 minutes of data received.
+ */
+@ApplicationAnnotation(name = "AutoComplete")
+public class AutoComplete implements StreamingApplication
+{
+
+  /**
+   * A dummy Twitter input operator. It reads from a text file containing some tweets and output a line every
+   * half of a second.
+   */
+  public static class TweetsInput extends BaseOperator implements InputOperator
+  {
+    private static boolean done = false;
+    public final transient DefaultOutputPort<String> output = new DefaultOutputPort<>();
+
+    private transient BufferedReader reader;
+  
+    public static boolean isDone()
+    {
+      return done;
+    }
+  
+    @Override
+    public void setup(OperatorContext context)
+    {
+      done = false;
+      initReader();
+    }
+
+    private void initReader()
+    {
+      try {
+        InputStream resourceStream = this.getClass().getResourceAsStream("/sampletweets.txt");
+        reader = new BufferedReader(new InputStreamReader(resourceStream));
+      } catch (Exception ex) {
+        throw Throwables.propagate(ex);
+      }
+    }
+
+    @Override
+    public void teardown()
+    {
+      IOUtils.closeQuietly(reader);
+    }
+
+    @Override
+    public void emitTuples()
+    {
+      try {
+        String line = reader.readLine();
+        if (line == null) {
+          done = true;
+          reader.close();
+          Thread.sleep(1000);
+        } else {
+          this.output.emit(line);
+        }
+        Thread.sleep(50);
+      } catch (IOException ex) {
+        throw new RuntimeException(ex);
+      } catch (InterruptedException e) {
+        // Ignore it.
+      }
+    }
+  }
+  
+  public static class Collector extends BaseOperator
+  {
+    private static Map<String, List<CompletionCandidate>> result = new HashMap<>();
+  
+    public static Map<String, List<CompletionCandidate>> getResult()
+    {
+      return result;
+    }
+  
+    public final transient DefaultInputPort<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>> input = new DefaultInputPort<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>>()
+    {
+      @Override
+      public void process(Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>> tuple)
+      {
+        result.put(tuple.getValue().getKey(), tuple.getValue().getValue());
+      }
+    };
+  }
+
+  /**
+   * FlapMap Function to extract all hashtags from a text form tweet.
+   */
+  private static class ExtractHashtags implements Function.FlatMapFunction<String, String>
+  {
+
+    @Override
+    public Iterable<String> f(String input)
+    {
+      List<String> result = new LinkedList<>();
+      Matcher m = Pattern.compile("#\\S+").matcher(input);
+      while (m.find()) {
+        result.add(m.group().substring(1));
+      }
+      return result;
+    }
+  }
+
+  /**
+   * Lower latency, but more expensive.
+   */
+  private static class ComputeTopFlat
+      extends CompositeStreamTransform<WindowedStream<CompletionCandidate>, WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>>>
+  {
+    private final int candidatesPerPrefix;
+    private final int minPrefix;
+
+    public ComputeTopFlat(int candidatesPerPrefix, int minPrefix)
+    {
+      this.candidatesPerPrefix = candidatesPerPrefix;
+      this.minPrefix = minPrefix;
+    }
+
+    @Override
+    public WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>> compose(
+        WindowedStream<CompletionCandidate> input)
+    {
+      return input
+        .<KeyValPair<String, CompletionCandidate>, WindowedStream<KeyValPair<String, CompletionCandidate>>>flatMap(new AllPrefixes(minPrefix))
+        .accumulateByKey(new TopNByKey(), new Function.ToKeyValue<KeyValPair<String, CompletionCandidate>, String,
+          CompletionCandidate>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, CompletionCandidate>> f(KeyValPair<String, CompletionCandidate> tuple)
+          {
+            // TODO: Should be removed after Auto-wrapping is supported. 
+            return new Tuple.WindowedTuple<>(Window.GLOBAL_WINDOW, tuple);
+          }
+        });
+    }
+  }
+
+  /**
+   * FlapMap Function to extract all prefixes of the hashtag in the input CompletionCandidate, and output
+   * KeyValPairs of the prefix and the CompletionCandidate
+   */
+  private static class AllPrefixes implements Function.FlatMapFunction<CompletionCandidate, KeyValPair<String, CompletionCandidate>>
+  {
+    private final int minPrefix;
+    private final int maxPrefix;
+
+    public AllPrefixes()
+    {
+      this(0, Integer.MAX_VALUE);
+    }
+
+    public AllPrefixes(int minPrefix)
+    {
+      this(minPrefix, Integer.MAX_VALUE);
+    }
+
+    public AllPrefixes(int minPrefix, int maxPrefix)
+    {
+      this.minPrefix = minPrefix;
+      this.maxPrefix = maxPrefix;
+    }
+
+    @Override
+    public Iterable<KeyValPair<String, CompletionCandidate>> f(CompletionCandidate input)
+    {
+      List<KeyValPair<String, CompletionCandidate>> result = new LinkedList<>();
+      String word = input.getValue();
+      for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
+
+        result.add(new KeyValPair<>(input.getValue().substring(0, i).toLowerCase(), input));
+      }
+      return result;
+    }
+  }
+
+  /**
+   * A Composite stream transform that takes as input a list of tokens and returns
+   * the most common tokens per prefix.
+   */
+  public static class ComputeTopCompletions
+      extends CompositeStreamTransform<WindowedStream<String>, WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>>>
+  {
+    private final int candidatesPerPrefix;
+    private final boolean recursive;
+
+    protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive)
+    {
+      this.candidatesPerPrefix = candidatesPerPrefix;
+      this.recursive = recursive;
+    }
+
+    public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive)
+    {
+      return new ComputeTopCompletions(candidatesPerPrefix, recursive);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>> compose(WindowedStream<String> inputStream)
+    {
+      ApexStream<CompletionCandidate> candidates = inputStream
+          .countByKey(new Function.ToKeyValue<String, String, Long>()
+          {
+            @Override
+            public Tuple<KeyValPair<String, Long>> f(String input)
+            {
+              return new Tuple.PlainTuple<>(new KeyValPair<>(input, 1L));
+            }
+          }, name("countByKey")).map(new Function.MapFunction<Tuple.WindowedTuple<KeyValPair<String,Long>>, CompletionCandidate>()
+          {
+            @Override
+            public CompletionCandidate f(Tuple.WindowedTuple<KeyValPair<String, Long>> input)
+            {
+              return new CompletionCandidate(input.getValue().getKey(), input.getValue().getValue());
+            }
+          }, name("ToCompletionCandidate"));
+
+      return candidates.addCompositeStreams(new ComputeTopFlat(10, 1));
+
+    }
+  }
+
+  /**
+   * Populate the dag with High-Level API.
+   * @param dag
+   * @param conf
+   */
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    TweetsInput input = new TweetsInput();
+    Collector collector = new Collector();
+
+    WindowOption windowOption = new WindowOption.GlobalWindow();
+
+    ApexStream<String> tags = StreamFactory.fromInput(input, input.output, name("tweetSampler"))
+        .flatMap(new ExtractHashtags());
+    
+    tags.window(windowOption, new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
+        .addCompositeStreams(ComputeTopCompletions.top(10, true)).endWith(collector, collector.input, name("collector"))
+        .populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/CompletionCandidate.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/CompletionCandidate.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/CompletionCandidate.java
new file mode 100644
index 0000000..8a7113e
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/CompletionCandidate.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+/**
+ * Class used to store tag-count pairs in Auto Complete Demo.
+ */
+public class CompletionCandidate implements Comparable<CompletionCandidate>
+{
+  private long count;
+  private String value;
+
+  public CompletionCandidate(String value, long count)
+  {
+    this.value = value;
+    this.count = count;
+  }
+
+  public long getCount()
+  {
+    return count;
+  }
+
+  public String getValue()
+  {
+    return value;
+  }
+
+  // Empty constructor required for Kryo.
+  public CompletionCandidate()
+  {
+    
+  }
+
+  @Override
+  public int compareTo(CompletionCandidate o)
+  {
+    if (this.count < o.count) {
+      return -1;
+    } else if (this.count == o.count) {
+      return this.value.compareTo(o.value);
+    } else {
+      return 1;
+    }
+  }
+
+  @Override
+  public boolean equals(Object other)
+  {
+    if (other instanceof CompletionCandidate) {
+      CompletionCandidate that = (CompletionCandidate)other;
+      return this.count == that.count && this.value.equals(that.value);
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Long.valueOf(count).hashCode() ^ value.hashCode();
+  }
+
+  @Override
+  public String toString()
+  {
+    return "CompletionCandidate[" + value + ", " + count + "]";
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/PojoEvent.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/PojoEvent.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/PojoEvent.java
new file mode 100644
index 0000000..2a4c003
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/PojoEvent.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.apex.malhar.stream.sample.complete;
+
+/**
+ * Tuple Class for JdbcOutput of StreamingWordExtract.
+ */
+public class PojoEvent extends Object
+{
+  private String stringValue;
+  
+  @Override
+  public String toString()
+  {
+    return "PojoEvent [stringValue=" + getStringValue() + "]";
+  }
+  
+  public void setStringValue(String newString)
+  {
+    this.stringValue = newString;
+  }
+  
+  public String getStringValue()
+  {
+    return this.stringValue;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtract.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtract.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtract.java
new file mode 100644
index 0000000..2ffdc82
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/StreamingWordExtract.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.Option;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.lib.db.jdbc.JdbcFieldInfo;
+import com.datatorrent.lib.db.jdbc.JdbcPOJOInsertOutputOperator;
+import com.datatorrent.lib.db.jdbc.JdbcTransactionalStore;
+
+import static java.sql.Types.VARCHAR;
+
+/**
+ * Beam StreamingWordExtract Example.
+ */
+@ApplicationAnnotation(name = "StreamingWordExtract")
+public class StreamingWordExtract implements StreamingApplication
+{
+  private static int wordCount = 0; // A counter to count number of words have been extracted.
+  private static int entriesMapped = 0; // A counter to count number of entries have been mapped.
+  
+  public int getWordCount()
+  {
+    return wordCount;
+  }
+  
+  public int getEntriesMapped()
+  {
+    return entriesMapped;
+  }
+  
+  /**
+   * A MapFunction that tokenizes lines of text into individual words.
+   */
+  public static class ExtractWords implements Function.FlatMapFunction<String, String>
+  {
+    @Override
+    public Iterable<String> f(String input)
+    {
+      List<String> result = new ArrayList<>(Arrays.asList(input.split("[^a-zA-Z0-9']+")));
+      wordCount += result.size();
+      return result;
+    }
+  }
+  
+  
+  /**
+   * A MapFunction that uppercases a word.
+   */
+  public static class Uppercase implements Function.MapFunction<String, String>
+  {
+    @Override
+    public String f(String input)
+    {
+      return input.toUpperCase();
+    }
+  }
+  
+  
+  /**
+   * A filter function to filter out empty strings.
+   */
+  public static class EmptyStringFilter implements Function.FilterFunction<String>
+  {
+    @Override
+    public boolean f(String input)
+    {
+      return !input.isEmpty();
+    }
+  }
+  
+  
+  /**
+   * A map function to map the result string to a pojo entry.
+   */
+  public static class PojoMapper implements Function.MapFunction<String, Object>
+  {
+  
+    @Override
+    public Object f(String input)
+    {
+      PojoEvent pojo = new PojoEvent();
+      pojo.setStringValue(input);
+      entriesMapped++;
+      return pojo;
+    }
+  }
+  
+  /**
+   * Add field infos to the {@link JdbcPOJOInsertOutputOperator}.
+   */
+  private static List<JdbcFieldInfo> addFieldInfos()
+  {
+    List<JdbcFieldInfo> fieldInfos = new ArrayList<>();
+    fieldInfos.add(new JdbcFieldInfo("STRINGVALUE", "stringValue", JdbcFieldInfo.SupportType.STRING, VARCHAR));
+    return fieldInfos;
+  }
+  
+  /**
+   * Populate dag with High-Level API.
+   * @param dag
+   * @param conf
+   */
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    JdbcPOJOInsertOutputOperator jdbcOutput = new JdbcPOJOInsertOutputOperator();
+    jdbcOutput.setFieldInfos(addFieldInfos());
+    JdbcTransactionalStore outputStore = new JdbcTransactionalStore();
+    jdbcOutput.setStore(outputStore);
+    jdbcOutput.setTablename("TestTable");
+    
+    // Create a stream reading from a folder.
+    ApexStream<String> stream = StreamFactory.fromFolder("./src/test/resources/data");
+
+    // Extract all the words from the input line of text.
+    stream.flatMap(new ExtractWords())
+      
+        // Filter out the empty strings.
+        .filter(new EmptyStringFilter())
+      
+        // Change every word to uppercase.
+        .map(new Uppercase())
+      
+        // Map the resulted word to a Pojo entry.
+        .map(new PojoMapper())
+      
+        // Output the entries to JdbcOutput and insert them into a table.
+        .endWith(jdbcOutput, jdbcOutput.input, Option.Options.name("jdbcOutput"));
+    
+    stream.populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopNByKey.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopNByKey.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopNByKey.java
new file mode 100644
index 0000000..a9e7744
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopNByKey.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * Specialized TopNByKey accumulation for AutoComplete Demo.
+ */
+public class TopNByKey implements
+    Accumulation<CompletionCandidate, Map<String, Long>, List<CompletionCandidate>>
+{
+  int n = 10;
+
+  Comparator comparator;
+
+  public void setN(int n)
+  {
+    this.n = n;
+  }
+
+  public void setComparator(Comparator comparator)
+  {
+    this.comparator = comparator;
+  }
+
+  @Override
+  public Map<String, Long> defaultAccumulatedValue()
+  {
+    return new HashMap<>();
+  }
+
+  /**
+   * Accumulate the input. Update the entry in the Accumulation Map if the key of the input is existed, create a
+   * new entry otherwise.
+   * @param accumulatedValue
+   * @param input
+   * @return
+   */
+  @Override
+  public Map<String, Long> accumulate(Map<String, Long> accumulatedValue, CompletionCandidate input)
+  {
+    accumulatedValue.put(input.getValue(), input.getCount());
+    return accumulatedValue;
+  }
+
+  /**
+   * Merge two Maps together. For every key, keep the larger value in the resulted Map.
+   * @param accumulatedValue1
+   * @param accumulatedValue2
+   * @return
+   */
+  @Override
+  public Map<String, Long> merge(Map<String, Long> accumulatedValue1, Map<String, Long> accumulatedValue2)
+  {
+    for (Map.Entry<String, Long> entry : accumulatedValue2.entrySet()) {
+      if (accumulatedValue1.containsKey(entry.getKey()) && accumulatedValue1.get(entry.getKey()) > entry.getValue()) {
+        continue;
+      }
+      accumulatedValue1.put(entry.getKey(), entry.getValue());
+    }
+    return accumulatedValue1;
+  }
+
+  /**
+   * Loop through the Accumulation Map to get the top n entries based on their values, return a list containing
+   * those entries.
+   * @param accumulatedValue
+   * @return
+   */
+  @Override
+  public List<CompletionCandidate> getOutput(Map<String, Long> accumulatedValue)
+  {
+    LinkedList<CompletionCandidate> result = new LinkedList<>();
+    for (Map.Entry<String, Long> entry : accumulatedValue.entrySet()) {
+      int k = 0;
+      for (CompletionCandidate inMemory : result) {
+        if (entry.getValue() > inMemory.getCount()) {
+          break;
+        }
+        k++;
+      }
+      result.add(k, new CompletionCandidate(entry.getKey(), entry.getValue()));
+      if (result.size() > n) {
+        result.remove(result.get(result.size() - 1));
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public List<CompletionCandidate> getRetraction(List<CompletionCandidate> value)
+  {
+    return new LinkedList<>();
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessions.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessions.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessions.java
new file mode 100644
index 0000000..de4e590
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TopWikipediaSessions.java
@@ -0,0 +1,340 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Random;
+import javax.annotation.Nullable;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.lib.window.impl.accumulation.TopN;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.hadoop.conf.Configuration;
+
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.lib.util.KeyValPair;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * Beam's TopWikipediaSessions Example.
+ */
+@ApplicationAnnotation(name = "TopWikipediaSessions")
+public class TopWikipediaSessions implements StreamingApplication
+{
+  /**
+   * A generator that outputs a stream of combinations of some users and some randomly generated edit time.
+   */
+  public static class SessionGen extends BaseOperator implements InputOperator
+  {
+    private String[] names = new String[]{"user1", "user2", "user3", "user4"};
+    public transient DefaultOutputPort<KeyValPair<String, Long>> output = new DefaultOutputPort<>();
+  
+    private static final Duration RAND_RANGE = Duration.standardDays(365);
+    private Long minTimestamp;
+    private long sleepTime;
+    private static int tupleCount = 0;
+  
+    public static int getTupleCount()
+    {
+      return tupleCount;
+    }
+  
+    private String randomName(String[] names)
+    {
+      int index = new Random().nextInt(names.length);
+      return names[index];
+    }
+  
+    @Override
+    public void setup(Context.OperatorContext context)
+    {
+      super.setup(context);
+      tupleCount = 0;
+      minTimestamp = System.currentTimeMillis();
+      sleepTime = context.getValue(Context.OperatorContext.SPIN_MILLIS);
+    }
+  
+    @Override
+    public void emitTuples()
+    {
+      long randMillis = (long)(Math.random() * RAND_RANGE.getMillis());
+      long randomTimestamp = minTimestamp + randMillis;
+      output.emit(new KeyValPair<String, Long>(randomName(names), randomTimestamp));
+      tupleCount++;
+      try {
+        Thread.sleep(sleepTime);
+      } catch (InterruptedException e) {
+        // Ignore it.
+      }
+    }
+  }
+  
+  public static class Collector extends BaseOperator
+  {
+    private final int resultSize = 5;
+    private static List<List<TempWrapper>> result = new ArrayList<>();
+  
+    public static List<List<TempWrapper>> getResult()
+    {
+      return result;
+    }
+  
+    public final transient DefaultInputPort<Tuple.WindowedTuple<List<TempWrapper>>> input = new DefaultInputPort<Tuple.WindowedTuple<List<TempWrapper>>>()
+    {
+      @Override
+      public void process(Tuple.WindowedTuple<List<TempWrapper>> tuple)
+      {
+        if (result.size() == resultSize) {
+          result.remove(0);
+        }
+        result.add(tuple.getValue());
+      }
+    };
+  }
+  
+  
+  /**
+   * Convert the upstream (user, time) combination to a timestamped tuple of user.
+   */
+  static class ExtractUserAndTimestamp implements Function.MapFunction<KeyValPair<String, Long>, Tuple.TimestampedTuple<String>>
+  {
+    @Override
+    public Tuple.TimestampedTuple<String> f(KeyValPair<String, Long> input)
+    {
+      long timestamp = input.getValue();
+      String userName = input.getKey();
+   
+      // Sets the implicit timestamp field to be used in windowing.
+      return new Tuple.TimestampedTuple<>(timestamp, userName);
+      
+    }
+  }
+  
+  /**
+   * Computes the number of edits in each user session.  A session is defined as
+   * a string of edits where each is separated from the next by less than an hour.
+   */
+  static class ComputeSessions
+      extends CompositeStreamTransform<ApexStream<Tuple.TimestampedTuple<String>>, WindowedStream<Tuple.WindowedTuple<KeyValPair<String, Long>>>>
+  {
+    @Override
+    public WindowedStream<Tuple.WindowedTuple<KeyValPair<String, Long>>> compose(ApexStream<Tuple.TimestampedTuple<String>> inputStream)
+    {
+      return inputStream
+        
+        // Chuck the stream into session windows.
+        .window(new WindowOption.SessionWindows(Duration.standardHours(1)), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
+        
+        // Count the number of edits for a user within one session.
+        .countByKey(new Function.ToKeyValue<Tuple.TimestampedTuple<String>, String, Long>()
+        {
+          @Override
+          public Tuple.TimestampedTuple<KeyValPair<String, Long>> f(Tuple.TimestampedTuple<String> input)
+          {
+            return new Tuple.TimestampedTuple<KeyValPair<String, Long>>(input.getTimestamp(), new KeyValPair<String, Long>(input.getValue(), 1L));
+          }
+        }, name("ComputeSessions"));
+    }
+  }
+  
+  /**
+   * A comparator class used for comparing two TempWrapper objects.
+   */
+  public static class Comp implements Comparator<TempWrapper>
+  {
+    @Override
+    public int compare(TempWrapper o1, TempWrapper o2)
+    {
+      return Long.compare(o1.getValue().getValue(), o2.getValue().getValue());
+    }
+  }
+  
+  /**
+   * A function to extract timestamp from a TempWrapper object.
+   */
+  // TODO: Need to revisit and change back to using TimestampedTuple.
+  public static class TimestampExtractor implements com.google.common.base.Function<TempWrapper, Long>
+  {
+    @Override
+    public Long apply(@Nullable TempWrapper input)
+    {
+      return input.getTimestamp();
+    }
+  }
+  
+  /**
+   * A temporary wrapper to wrap a KeyValPair and a timestamp together to represent a timestamped tuple, the reason
+   * for this is that we cannot resolve a type conflict when calling accumulate(). After the issue resolved, we can
+   * remove this class.
+   */
+  public static class TempWrapper
+  {
+    private KeyValPair<String, Long> value;
+    private Long timestamp;
+    
+    public TempWrapper()
+    {
+      
+    }
+    
+    public TempWrapper(KeyValPair<String, Long> value, Long timestamp)
+    {
+      this.value = value;
+      this.timestamp = timestamp;
+    }
+  
+    @Override
+    public String toString()
+    {
+      return this.value + "  -  " + this.timestamp;
+    }
+  
+    public Long getTimestamp()
+    {
+      return timestamp;
+    }
+  
+    public void setTimestamp(Long timestamp)
+    {
+      this.timestamp = timestamp;
+    }
+  
+    public KeyValPair<String, Long> getValue()
+    {
+      return value;
+    }
+  
+    public void setValue(KeyValPair<String, Long> value)
+    {
+      this.value = value;
+    }
+  }
+
+  /**
+   * Computes the longest session ending in each month, in this case we use 30 days to represent every month.
+   */
+  private static class TopPerMonth
+      extends CompositeStreamTransform<ApexStream<Tuple.WindowedTuple<KeyValPair<String, Long>>>, WindowedStream<Tuple.WindowedTuple<List<TempWrapper>>>>
+  {
+    
+    @Override
+    public WindowedStream<Tuple.WindowedTuple<List<TempWrapper>>> compose(ApexStream<Tuple.WindowedTuple<KeyValPair<String, Long>>> inputStream)
+    {
+      TopN<TempWrapper> topN = new TopN<>();
+      topN.setN(10);
+      topN.setComparator(new Comp());
+      
+      return inputStream
+        
+        // Map the input WindowedTuple to a TempWrapper object.
+        .map(new Function.MapFunction<Tuple.WindowedTuple<KeyValPair<String, Long>>, TempWrapper>()
+        {
+          @Override
+          public TempWrapper f(Tuple.WindowedTuple<KeyValPair<String, Long>> input)
+          {
+            return new TempWrapper(input.getValue(), input.getWindows().get(0).getBeginTimestamp());
+          }
+        }, name("TempWrapper"))
+        
+        // Apply window and trigger option again, this time chuck the stream into fixed time windows.
+        .window(new WindowOption.TimeWindows(Duration.standardDays(30)), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(Duration.standardSeconds(5)))
+        
+        // Compute the top 10 user-sessions with most number of edits.
+        .accumulate(topN, name("TopN")).with("timestampExtractor", new TimestampExtractor());
+    }
+  }
+  
+  /**
+   * A map function that combine the user and his/her edit session together to a string and use that string as a key
+   * with number of edits in that session as value to create a new key value pair to send to downstream.
+   */
+  static class SessionsToStringsDoFn implements Function.MapFunction<Tuple.WindowedTuple<KeyValPair<String, Long>>, Tuple.WindowedTuple<KeyValPair<String, Long>>>
+  {
+    @Override
+    public Tuple.WindowedTuple<KeyValPair<String, Long>> f(Tuple.WindowedTuple<KeyValPair<String, Long>> input)
+    {
+      return new Tuple.WindowedTuple<KeyValPair<String, Long>>(input.getWindows().get(0), new KeyValPair<String, Long>(
+        input.getValue().getKey()  + " : " + input.getWindows().get(0).getBeginTimestamp() + " : " + input.getWindows().get(0).getDurationMillis(),
+        input.getValue().getValue()));
+    }
+  }
+  
+  /**
+   * A flapmap function that turns the result into readable format.
+   */
+  static class FormatOutputDoFn implements Function.FlatMapFunction<Tuple.WindowedTuple<List<TempWrapper>>, String>
+  {
+    @Override
+    public Iterable<String> f(Tuple.WindowedTuple<List<TempWrapper>> input)
+    {
+      ArrayList<String> result = new ArrayList<>();
+      for (TempWrapper item : input.getValue()) {
+        String session = item.getValue().getKey();
+        long count = item.getValue().getValue();
+        result.add(session + " + " + count + " : " + input.getWindows().get(0).getBeginTimestamp());
+      }
+      return result;
+    }
+  }
+  
+  /**
+   * A composite transform that compute the top wikipedia sessions.
+   */
+  public static class ComputeTopSessions extends CompositeStreamTransform<ApexStream<KeyValPair<String, Long>>, WindowedStream<Tuple.WindowedTuple<List<TempWrapper>>>>
+  {
+    @Override
+    public WindowedStream<Tuple.WindowedTuple<List<TempWrapper>>> compose(ApexStream<KeyValPair<String, Long>> inputStream)
+    {
+      return inputStream
+        .map(new ExtractUserAndTimestamp(), name("ExtractUserAndTimestamp"))
+        .addCompositeStreams(new ComputeSessions())
+        .map(new SessionsToStringsDoFn(), name("SessionsToStringsDoFn"))
+        .addCompositeStreams(new TopPerMonth());
+    }
+  }
+  
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    SessionGen sg = new SessionGen();
+    Collector collector = new Collector();
+    StreamFactory.fromInput(sg, sg.output, name("sessionGen"))
+      .addCompositeStreams(new ComputeTopSessions())
+      .endWith(collector, collector.input, name("collector")).populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutes.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutes.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutes.java
new file mode 100644
index 0000000..2cc04d1
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TrafficRoutes.java
@@ -0,0 +1,521 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.WindowOption;
+import org.apache.apex.malhar.lib.window.impl.accumulation.Group;
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.collect.Lists;
+
+import com.datatorrent.api.Context;
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.common.util.BaseOperator;
+import com.datatorrent.lib.util.KeyValPair;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * Beam's TrafficRoutes example.
+ */
+@ApplicationAnnotation(name = "TrafficRoutes")
+public class TrafficRoutes implements StreamingApplication
+{
+  static Map<String, String> sdStations = buildStationInfo();
+  static final int WINDOW_DURATION = 3;  // Default sliding window duration in minutes
+  static final int WINDOW_SLIDE_EVERY = 1;  // Default window 'slide every' setting in minutes
+  
+  /**
+   * This class holds information about a station reading's average speed.
+   */
+  public static class StationSpeed implements Comparable<StationSpeed>
+  {
+    @Nullable
+    String stationId;
+    @Nullable
+    Double avgSpeed;
+    @Nullable
+    Long timestamp;
+    
+    public StationSpeed() {}
+    
+    public StationSpeed(String stationId, Double avgSpeed, Long timestamp)
+    {
+      this.stationId = stationId;
+      this.avgSpeed = avgSpeed;
+      this.timestamp = timestamp;
+    }
+  
+    public void setAvgSpeed(@Nullable Double avgSpeed)
+    {
+      this.avgSpeed = avgSpeed;
+    }
+  
+    public void setStationId(@Nullable String stationId)
+    {
+      this.stationId = stationId;
+    }
+  
+    public void setTimestamp(@Nullable Long timestamp)
+    {
+      this.timestamp = timestamp;
+    }
+  
+    @Nullable
+    public Long getTimestamp()
+    {
+      return timestamp;
+    }
+  
+    public String getStationId()
+    {
+      return this.stationId;
+    }
+    
+    public Double getAvgSpeed()
+    {
+      return this.avgSpeed;
+    }
+    
+    @Override
+    public int compareTo(StationSpeed other)
+    {
+      return Long.compare(this.timestamp, other.timestamp);
+    }
+  }
+  
+  /**
+   * This class holds information about a route's speed/slowdown.
+   */
+  static class RouteInfo
+  {
+    @Nullable
+    String route;
+    @Nullable
+    Double avgSpeed;
+    @Nullable
+    Boolean slowdownEvent;
+    
+    public RouteInfo()
+    {
+      
+    }
+    
+    public RouteInfo(String route, Double avgSpeed, Boolean slowdownEvent)
+    {
+      this.route = route;
+      this.avgSpeed = avgSpeed;
+      this.slowdownEvent = slowdownEvent;
+    }
+    
+    public String getRoute()
+    {
+      return this.route;
+    }
+    
+    public Double getAvgSpeed()
+    {
+      return this.avgSpeed;
+    }
+    
+    public Boolean getSlowdownEvent()
+    {
+      return this.slowdownEvent;
+    }
+  }
+  
+  /**
+   * Extract the timestamp field from the input string, and wrap the input string in a {@link Tuple.TimestampedTuple}
+   * with the extracted timestamp.
+   */
+  static class ExtractTimestamps implements Function.MapFunction<String, Tuple.TimestampedTuple<String>>
+  {
+    
+    @Override
+    public Tuple.TimestampedTuple<String> f(String input)
+    {
+      String[] items = input.split(",");
+      String timestamp = tryParseTimestamp(items);
+    
+      return new Tuple.TimestampedTuple<>(Long.parseLong(timestamp), input);
+    }
+  }
+  
+  /**
+   * Filter out readings for the stations along predefined 'routes', and output
+   * (station, speed info) keyed on route.
+   */
+  static class ExtractStationSpeedFn implements Function.FlatMapFunction<Tuple.TimestampedTuple<String>, KeyValPair<String, StationSpeed>>
+  {
+    
+    @Override
+    public Iterable<KeyValPair<String, StationSpeed>> f(Tuple.TimestampedTuple<String> input)
+    {
+      
+      ArrayList<KeyValPair<String, StationSpeed>> result = new ArrayList<>();
+      String[] items = input.getValue().split(",");
+      String stationType = tryParseStationType(items);
+      // For this analysis, use only 'main line' station types
+      if (stationType != null && stationType.equals("ML")) {
+        Double avgSpeed = tryParseAvgSpeed(items);
+        String stationId = tryParseStationId(items);
+        // For this simple example, filter out everything but some hardwired routes.
+        if (avgSpeed != null && stationId != null && sdStations.containsKey(stationId)) {
+          StationSpeed stationSpeed =
+              new StationSpeed(stationId, avgSpeed, input.getTimestamp());
+          // The tuple key is the 'route' name stored in the 'sdStations' hash.
+          KeyValPair<String, StationSpeed> outputValue = new KeyValPair<>(sdStations.get(stationId), stationSpeed);
+          result.add(outputValue);
+        }
+      }
+      return result;
+    }
+  }
+  
+  /**
+   * For a given route, track average speed for the window. Calculate whether
+   * traffic is currently slowing down, via a predefined threshold. If a supermajority of
+   * speeds in this sliding window are less than the previous reading we call this a 'slowdown'.
+   * Note: these calculations are for example purposes only, and are unrealistic and oversimplified.
+   */
+  static class GatherStats
+      implements Function.FlatMapFunction<Tuple.WindowedTuple<KeyValPair<String, List<StationSpeed>>>, Tuple.TimestampedTuple<KeyValPair<String, RouteInfo>>>
+  {
+    @Override
+    public Iterable<Tuple.TimestampedTuple<KeyValPair<String, RouteInfo>>> f(Tuple.WindowedTuple<KeyValPair<String, List<StationSpeed>>> input)
+    {
+      ArrayList<Tuple.TimestampedTuple<KeyValPair<String, RouteInfo>>> result = new ArrayList<>();
+      String route = input.getValue().getKey();
+      double speedSum = 0.0;
+      int speedCount = 0;
+      int speedups = 0;
+      int slowdowns = 0;
+      List<StationSpeed> infoList = Lists.newArrayList(input.getValue().getValue());
+      // StationSpeeds sort by embedded timestamp.
+      Collections.sort(infoList);
+      Map<String, Double> prevSpeeds = new HashMap<>();
+      // For all stations in the route, sum (non-null) speeds. Keep a count of the non-null speeds.
+      for (StationSpeed item : infoList) {
+        Double speed = item.getAvgSpeed();
+        if (speed != null) {
+          speedSum += speed;
+          speedCount++;
+          Double lastSpeed = prevSpeeds.get(item.getStationId());
+          if (lastSpeed != null) {
+            if (lastSpeed < speed) {
+              speedups += 1;
+            } else {
+              slowdowns += 1;
+            }
+          }
+          prevSpeeds.put(item.getStationId(), speed);
+        }
+      }
+      if (speedCount == 0) {
+        // No average to compute.
+        return result;
+      }
+      double speedAvg = speedSum / speedCount;
+      boolean slowdownEvent = slowdowns >= 2 * speedups;
+      RouteInfo routeInfo = new RouteInfo(route, speedAvg, slowdownEvent);
+      result.add(new Tuple.TimestampedTuple<KeyValPair<String, RouteInfo>>(input.getTimestamp(), new KeyValPair<String, RouteInfo>(route, routeInfo)));
+      return result;
+    }
+  }
+  
+  /**
+   * Output Pojo class for outputting result to JDBC.
+   */
+  static class OutputPojo
+  {
+    private Double avgSpeed;
+    private Boolean slowdownEvent;
+    private String key;
+    private Long timestamp;
+    
+    public OutputPojo()
+    {
+    }
+ 
+    public OutputPojo(Double avgSpeed, Boolean slowdownEvent, String key, Long timestamp)
+    {
+      this.avgSpeed = avgSpeed;
+      this.slowdownEvent = slowdownEvent;
+      this.key = key;
+      this.timestamp = timestamp;
+    }
+  
+    @Override
+    public String toString()
+    {
+      return key + " + " + avgSpeed + " + " + slowdownEvent + " + " + timestamp;
+    }
+  
+    public void setTimestamp(Long timestamp)
+    {
+      this.timestamp = timestamp;
+    }
+  
+    public Long getTimestamp()
+    {
+      return timestamp;
+    }
+  
+    public void setAvgSpeed(Double avgSpeed)
+    {
+      this.avgSpeed = avgSpeed;
+    }
+  
+    public Double getAvgSpeed()
+    {
+      return avgSpeed;
+    }
+  
+    public void setKey(String key)
+    {
+      this.key = key;
+    }
+  
+    public String getKey()
+    {
+      return key;
+    }
+  
+    public void setSlowdownEvent(Boolean slowdownEvent)
+    {
+      this.slowdownEvent = slowdownEvent;
+    }
+  
+    public Boolean getSlowdownEvent()
+    {
+      return slowdownEvent;
+    }
+    
+  }
+  
+  public static class Collector extends BaseOperator
+  {
+    private static Map<KeyValPair<Long, String>, KeyValPair<Double, Boolean>> result = new HashMap<>();
+  
+    public static Map<KeyValPair<Long, String>, KeyValPair<Double, Boolean>> getResult()
+    {
+      return result;
+    }
+    
+    public final transient DefaultInputPort<OutputPojo> input = new DefaultInputPort<OutputPojo>()
+    {
+      @Override
+      public void process(OutputPojo tuple)
+      {
+        result.put(new KeyValPair<Long, String>(tuple.getTimestamp(), tuple.getKey()), new KeyValPair<Double, Boolean>(tuple.getAvgSpeed(), tuple.getSlowdownEvent()));
+      }
+    };
+  }
+  
+  /**
+   * Format the results of the slowdown calculations to a OutputPojo.
+   */
+  static class FormatStatsFn implements Function.MapFunction<Tuple.TimestampedTuple<KeyValPair<String, RouteInfo>>, OutputPojo>
+  {
+    @Override
+    public OutputPojo f(Tuple.TimestampedTuple<KeyValPair<String, RouteInfo>> input)
+    {
+      RouteInfo routeInfo = input.getValue().getValue();
+      OutputPojo row = new OutputPojo(routeInfo.getAvgSpeed(), routeInfo.getSlowdownEvent(), input.getValue().getKey(), input.getTimestamp());
+      return row;
+    }
+  }
+    
+  
+  /**
+   * This composite transformation extracts speed info from traffic station readings.
+   * It groups the readings by 'route' and analyzes traffic slowdown for that route.
+   * Lastly, it formats the results for JDBC.
+   */
+  static class TrackSpeed extends
+      CompositeStreamTransform<WindowedStream<KeyValPair<String, StationSpeed>>, WindowedStream<OutputPojo>>
+  {
+    @Override
+    public WindowedStream<OutputPojo> compose(WindowedStream<KeyValPair<String, StationSpeed>> inputStream)
+    {
+      // Apply a GroupByKey transform to collect a list of all station
+      // readings for a given route.
+      WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<StationSpeed>>>> timeGroup =
+          inputStream
+          .accumulateByKey(new Group<StationSpeed>(), new Function.ToKeyValue<KeyValPair<String, StationSpeed>, String, StationSpeed>()
+          {
+            @Override
+            public Tuple<KeyValPair<String, StationSpeed>> f(KeyValPair<String, StationSpeed> input)
+            {
+              return new Tuple.TimestampedTuple<>(input.getValue().getTimestamp(), input);
+            }
+          }, name("GroupByKey"));
+      
+      // Analyze 'slowdown' over the route readings.
+      WindowedStream<Tuple.TimestampedTuple<KeyValPair<String, RouteInfo>>> stats = timeGroup
+          .flatMap(new GatherStats(), name("GatherStats"));
+      
+      // Format the results for writing to JDBC table.
+      WindowedStream<OutputPojo> results = stats.map(new FormatStatsFn(), name("FormatStatsFn"));
+      
+      return results;
+    }
+  }
+  
+  
+  private static Double tryParseAvgSpeed(String[] inputItems)
+  {
+    try {
+      return Double.parseDouble(tryParseString(inputItems, 3));
+    } catch (NumberFormatException e) {
+      return null;
+    } catch (NullPointerException e) {
+      return null;
+    }
+  }
+  
+  private static String tryParseStationType(String[] inputItems)
+  {
+    return tryParseString(inputItems, 2);
+  }
+  
+  private static String tryParseStationId(String[] inputItems)
+  {
+    return tryParseString(inputItems, 1);
+  }
+  
+  private static String tryParseTimestamp(String[] inputItems)
+  {
+    return tryParseString(inputItems, 0);
+  }
+  
+  private static String tryParseString(String[] inputItems, int index)
+  {
+    return inputItems.length >= index ? inputItems[index] : null;
+  }
+  
+  /**
+   * Define some small hard-wired San Diego 'routes' to track based on sensor station ID.
+   */
+  private static Map<String, String> buildStationInfo()
+  {
+    Map<String, String> stations = new Hashtable<String, String>();
+    stations.put("1108413", "SDRoute1"); // from freeway 805 S
+    stations.put("1108699", "SDRoute2"); // from freeway 78 E
+    stations.put("1108702", "SDRoute2");
+    return stations;
+  }
+  
+  /**
+   * A dummy generator to generate some traffic information.
+   */
+  public static class InfoGen extends BaseOperator implements InputOperator
+  {
+    public transient DefaultOutputPort<String> output = new DefaultOutputPort<>();
+    
+    private String[] stationTypes = new String[]{"ML", "BL", "GL"};
+    private int[] stationIDs = new int[]{1108413, 1108699, 1108702};
+    private double ave = 55.0;
+    private long timestamp;
+    private static final Duration RAND_RANGE = Duration.standardMinutes(10);
+    private static int tupleCount = 0;
+  
+    public static int getTupleCount()
+    {
+      return tupleCount;
+    }
+  
+    @Override
+    public void setup(Context.OperatorContext context)
+    {
+      tupleCount = 0;
+      timestamp = System.currentTimeMillis();
+    }
+  
+    @Override
+    public void emitTuples()
+    {
+      for (String stationType : stationTypes) {
+        for (int stationID : stationIDs) {
+          double speed = Math.random() * 20 + ave;
+          long time = (long)(Math.random() * RAND_RANGE.getMillis()) + timestamp;
+          try {
+            output.emit(time + "," + stationID + "," + stationType + "," + speed);
+            tupleCount++;
+         
+            Thread.sleep(50);
+          } catch (Exception e) {
+            // Ignore it
+          }
+        }
+      }
+    }
+  }
+  
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    InfoGen infoGen = new InfoGen();
+    Collector collector = new Collector();
+    
+    // Create a stream from the input operator.
+    ApexStream<Tuple.TimestampedTuple<String>> stream = StreamFactory.fromInput(infoGen, infoGen.output, name("infoGen"))
+        
+        // Extract the timestamp from the input and wrap it into a TimestampedTuple.
+        .map(new ExtractTimestamps(), name("ExtractTimestamps"));
+    
+    stream
+        // Extract the average speed of a station.
+        .flatMap(new ExtractStationSpeedFn(), name("ExtractStationSpeedFn"))
+      
+        // Apply window and trigger option.
+        .window(new WindowOption.SlidingTimeWindows(Duration.standardMinutes(WINDOW_DURATION), Duration.standardMinutes(WINDOW_SLIDE_EVERY)), new TriggerOption().withEarlyFiringsAtEvery(Duration.millis(5000)).accumulatingFiredPanes())
+        
+        // Apply TrackSpeed composite transformation to compute the route information.
+        .addCompositeStreams(new TrackSpeed())
+      
+        // print the result to console.
+        .print()
+        .endWith(collector, collector.input, name("Collector"))
+        .populateDag(dag);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoComplete.java
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoComplete.java b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoComplete.java
new file mode 100644
index 0000000..ecad622
--- /dev/null
+++ b/demos/highlevelapi/src/main/java/org/apache/apex/malhar/stream/sample/complete/TwitterAutoComplete.java
@@ -0,0 +1,251 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.stream.sample.complete;
+
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetEncoder;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.joda.time.Duration;
+
+import org.apache.apex.malhar.lib.window.TriggerOption;
+import org.apache.apex.malhar.lib.window.Tuple;
+import org.apache.apex.malhar.lib.window.Window;
+import org.apache.apex.malhar.lib.window.WindowOption;
+
+import org.apache.apex.malhar.stream.api.ApexStream;
+import org.apache.apex.malhar.stream.api.CompositeStreamTransform;
+import org.apache.apex.malhar.stream.api.WindowedStream;
+import org.apache.apex.malhar.stream.api.function.Function;
+import org.apache.apex.malhar.stream.api.impl.StreamFactory;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.datatorrent.api.DAG;
+import com.datatorrent.api.StreamingApplication;
+import com.datatorrent.api.annotation.ApplicationAnnotation;
+import com.datatorrent.contrib.twitter.TwitterSampleInput;
+import com.datatorrent.lib.util.KeyValPair;
+
+import static org.apache.apex.malhar.stream.api.Option.Options.name;
+
+/**
+ * Auto Complete Hashtag Demo with real time twitter input. In order to run this application, you need to create an app
+ * at https://apps.twitter.com, then generate your consumer and access keys and tokens, and enter those information
+ * accordingly in /resources/META-INF/properties.xml.
+ *
+ * The authentication requires following 4 information.
+ * Your application consumer key,
+ * Your application consumer secret,
+ * Your twitter access token, and
+ * Your twitter access token secret.
+ */
+@ApplicationAnnotation(name = "TwitterAutoComplete")
+public class TwitterAutoComplete implements StreamingApplication
+{
+  /**
+   * Check whether every character in a string is ASCII encoding.
+   */
+  public static class StringUtils
+  {
+    static CharsetEncoder encoder = Charset.forName("US-ASCII").newEncoder();
+
+    public static boolean isAscii(String v)
+    {
+      return encoder.canEncode(v);
+    }
+  }
+
+  /**
+   * FlapMap Function to extract all hashtags from a text form tweet.
+   */
+  private static class ExtractHashtags implements Function.FlatMapFunction<String, String>
+  {
+
+    @Override
+    public Iterable<String> f(String input)
+    {
+      List<String> result = new LinkedList<>();
+      Matcher m = Pattern.compile("#\\S+").matcher(input);
+      while (m.find()) {
+        result.add(m.group().substring(1));
+      }
+      return result;
+    }
+  }
+
+  /**
+   * Lower latency, but more expensive.
+   */
+  private static class ComputeTopFlat
+      extends CompositeStreamTransform<WindowedStream<CompletionCandidate>, WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>>>
+  {
+    private final int candidatesPerPrefix;
+    private final int minPrefix;
+
+    public ComputeTopFlat(int candidatesPerPrefix, int minPrefix)
+    {
+      this.candidatesPerPrefix = candidatesPerPrefix;
+      this.minPrefix = minPrefix;
+    }
+
+    @Override
+    public WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>> compose(
+        WindowedStream<CompletionCandidate> input)
+    {
+      TopNByKey topNByKey = new TopNByKey();
+      topNByKey.setN(candidatesPerPrefix);
+      return input
+        .<KeyValPair<String, CompletionCandidate>, WindowedStream<KeyValPair<String, CompletionCandidate>>>flatMap(new AllPrefixes(minPrefix, 3), name("Extract Prefixes"))
+        .accumulateByKey(topNByKey, new Function.ToKeyValue<KeyValPair<String, CompletionCandidate>, String, CompletionCandidate>()
+        {
+          @Override
+          public Tuple<KeyValPair<String, CompletionCandidate>> f(KeyValPair<String, CompletionCandidate> tuple)
+          {
+            // TODO: Should be removed after Auto-wrapping is supported.
+            return new Tuple.WindowedTuple<>(Window.GLOBAL_WINDOW, tuple);
+          }
+        }, name("TopNByKey"));
+    }
+  }
+
+  /**
+   * FlapMap Function to extract all prefixes of the hashtag in the input CompletionCandidate, and output
+   * KeyValPairs of the prefix and the CompletionCandidate
+   */
+  private static class AllPrefixes implements Function.FlatMapFunction<CompletionCandidate, KeyValPair<String, CompletionCandidate>>
+  {
+    private final int minPrefix;
+    private final int maxPrefix;
+
+    public AllPrefixes()
+    {
+      this(0, Integer.MAX_VALUE);
+    }
+
+    public AllPrefixes(int minPrefix)
+    {
+      this(minPrefix, Integer.MAX_VALUE);
+    }
+
+    public AllPrefixes(int minPrefix, int maxPrefix)
+    {
+      this.minPrefix = minPrefix;
+      this.maxPrefix = maxPrefix;
+    }
+
+    @Override
+    public Iterable<KeyValPair<String, CompletionCandidate>> f(CompletionCandidate input)
+    {
+      List<KeyValPair<String, CompletionCandidate>> result = new LinkedList<>();
+      String word = input.getValue();
+      for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
+        result.add(new KeyValPair<>(input.getValue().substring(0, i).toLowerCase(), input));
+      }
+      return result;
+    }
+  }
+
+  /**
+   * A Composite stream transform that takes as input a list of tokens and returns
+   * the most common tokens per prefix.
+   */
+  public static class ComputeTopCompletions
+      extends CompositeStreamTransform<WindowedStream<String>, WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>>>
+  {
+    private final int candidatesPerPrefix;
+    private final boolean recursive;
+
+    protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive)
+    {
+      this.candidatesPerPrefix = candidatesPerPrefix;
+      this.recursive = recursive;
+    }
+
+    public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive)
+    {
+      return new ComputeTopCompletions(candidatesPerPrefix, recursive);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public WindowedStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>> compose(WindowedStream<String> inputStream)
+    {
+
+      ApexStream<CompletionCandidate> candidates = inputStream
+          .countByKey(new Function.ToKeyValue<String, String, Long>()
+          {
+            @Override
+            public Tuple<KeyValPair<String, Long>> f(String input)
+            {
+              return new Tuple.PlainTuple<>(new KeyValPair<>(input, 1L));
+            }
+          }, name("Hashtag Count")).map(new Function.MapFunction<Tuple.WindowedTuple<KeyValPair<String,Long>>, CompletionCandidate>()
+          {
+            @Override
+            public CompletionCandidate f(Tuple.WindowedTuple<KeyValPair<String, Long>> input)
+            {
+              return new CompletionCandidate(input.getValue().getKey(), input.getValue().getValue());
+            }
+          }, name("KeyValPair to CompletionCandidate"));
+
+      return candidates.addCompositeStreams(new ComputeTopFlat(candidatesPerPrefix, 1));
+
+    }
+  }
+
+  /**
+   * FilterFunction to filter out tweets with non-acsii characters.
+   */
+  static class ASCIIFilter implements Function.FilterFunction<String>
+  {
+    @Override
+    public boolean f(String input)
+    {
+      return StringUtils.isAscii(input);
+    }
+  }
+
+  /**
+   * Populate the dag with High-Level API.
+   * @param dag
+   * @param conf
+   */
+  @Override
+  public void populateDAG(DAG dag, Configuration conf)
+  {
+    TwitterSampleInput input = new TwitterSampleInput();
+
+    WindowOption windowOption = new WindowOption.GlobalWindow();
+
+    ApexStream<String> tags = StreamFactory.fromInput(input, input.text, name("tweetSampler"))
+        .filter(new ASCIIFilter(), name("ACSII Filter"))
+        .flatMap(new ExtractHashtags(), name("Extract Hashtags"));
+
+    ApexStream<Tuple.WindowedTuple<KeyValPair<String, List<CompletionCandidate>>>> s =
+        tags.window(windowOption, new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(Duration.standardSeconds(10)))
+        .addCompositeStreams(ComputeTopCompletions.top(10, true)).print();
+
+    s.populateDag(dag);
+  }
+}


[2/6] apex-malhar git commit: Added Beam Examples and Implementations of Accumulation.

Posted by th...@apache.org.
http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/resources/log4j.properties b/demos/highlevelapi/src/test/resources/log4j.properties
new file mode 100644
index 0000000..592eb19
--- /dev/null
+++ b/demos/highlevelapi/src/test/resources/log4j.properties
@@ -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.
+#
+
+log4j.rootLogger=INFO,CONSOLE
+
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n
+log4j.appender.CONSOLE.threshold=INFO
+#log4j.appender.CONSOLE.threshold=${test.log.console.threshold}
+test.log.console.threshold=WARN
+
+log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} %M - %m%n
+log4j.appender.RFA.File=/tmp/app.log
+
+# to enable, add SYSLOG to rootLogger
+log4j.appender.SYSLOG=org.apache.log4j.net.SyslogAppender
+log4j.appender.SYSLOG.syslogHost=127.0.0.1
+log4j.appender.SYSLOG.layout=org.apache.log4j.PatternLayout
+log4j.appender.SYSLOG.layout.conversionPattern=${dt.cid} %-5p [%t] %c{2} %x - %m%n
+log4j.appender.SYSLOG.Facility=LOCAL1
+
+#log4j.logger.org=INFO
+
+#log4j.logger.org.apache.commons.beanutils=warn
+log4j.logger.com.datatorrent=INFO
+log4j.logger.org.apache.apex=INFO

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/resources/sampletweets.txt
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/resources/sampletweets.txt b/demos/highlevelapi/src/test/resources/sampletweets.txt
new file mode 100644
index 0000000..475fada
--- /dev/null
+++ b/demos/highlevelapi/src/test/resources/sampletweets.txt
@@ -0,0 +1,207 @@
+Tweet content
+"Apple has $233 billion in cash. It could buy all
+
+\u2014@NFL teams
+\u2014@NBA teams
+\u2014@MLB teams
+\u2014@NHL teams
+
+...and still have $80 billion left. $AAPL"
+Read $MRNJ #NEWS, $HEMP &amp; $GRCU r above .01, that's where we r goin\U0001f680 $SPY $MSFT $SBUX $SFOR $VRX $AAPL $TSLA $GOOG $FB $EURUSD $USDJPY $MLCG
+$RXSF is leadin their sector accordin 2 @EdisonMediaCen $AAPL $SPY $TSLA $FB $EURUSD $ALK $IBB $EW $AMZN $GBPUSD $GM https://t.co/LYY2mHn755
+Philstockworld Top Trade Review $AAPL $MSFT #Dividends $USO $HOV $TWTR -- https://t.co/JArXsIm7CI https://t.co/kRR9ezhm9E
+Philstockworld Top Trade Review: $AAPL $ABX $BA $CAKE $CMG $DIS $IBM $GILD $LL $UNG $SPY -- https://t.co/EX5SYjdwBC https://t.co/7FBZwVZ63v
+"Monday\u2019s Oil Mess: Rent-A-Rebel Jacks up Prices into the Holiday $USO $AAPL
+#Earnings -- https://t.co/cGHB3WDKA8 https://t.co/JFZIBcom1n"
+Meaningless Monday Market Movement! $AAPL $SQQQ #oil #Brexit https://t.co/j4Iqg7E1HN
+"S&amp;P Futures Back over 2,050, for Now
+$SPY $AAPL $SQQQ #China #Debt #Hedging -- https://t.co/2dOc5T89S3 https://t.co/TDPVdNRNQF"
+"\U0001f4a5TURN YOUR $500 INTO $5,000+\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE\u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $AAPL $GSAT $MGT
+https://t.co/lwAGjfmIP3"
+Trendless Tuesday - Watch Yesterday\u2019s Fake Gains Disappear $AAPL #China $FXI #Earnings -- https://t.co/GpgGqoOlFn https://t.co/FRuixv5aZF
+"\U0001f4a5TURN YOUR $500 INTO $5,000+\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE\u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $AAPL $UVXY $JDST
+https://t.co/lwAGjfmIP3"
+"Apple has $233 billion in cash. It could buy:
+
+Uber
+Tesla
+Twitter
+Airbnb
+Netflix
+Yahoo
+
+...and still have $18 billion left. $AAPL"
+Option Opportunity Portfolio May Review \u2013 Up 19.3% In 30 Days! $ABX $FCX $USO $AAPL $DIS - https://t.co/rp3kMsRZ3E https://t.co/TKkc15pKcR
+Waiting for the Fed \u2013 Apple Gives Us Huge Wins: $AAPL $SQQQ #GDP #Nikkei #Futures #Oil -- https://t.co/Al3pkf350V https://t.co/LktIRF4F2b
+Tempting Tuesday - S&amp;P 2,100 is Still the Line to Watch Ahead of the Fed $AAPL $QQQ -- https://t.co/t1eDfKHJnk https://t.co/BAW3RAe7SC
+Our $SQQQ Hedge is Up 314% and Our Futures Are Up $4,850, You're Welcome!  $AAPL -- https://t.co/eUQ2kCkCOY https://t.co/Yk98oyqMZl
+"TURN YOUR \U0001f4b2500 INTO \U0001f4b25,000$\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE \u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $TWTR $AAPL $LNKD
+https://t.co/euJFNQX1g4"
+"TURN YOUR \U0001f4b2500 INTO \U0001f4b25,000$\U0001f4a5
+
+JOIN #TEAMBILLIONAIRE \u2935
+\U0001f4e7 pennystockhotline@gmail.com
+
+#PENNYSTOCKS $TALK $PPPI $AAPL https://t.co/oSn11kxftM"
+Bears today. We getting paid! $AAPL $TWTR $BWLD $NFLX https://t.co/CCi0S3skJJ
+"Apple has $233 billion in cash. It could buy all
+
+\u2014@NFL teams
+\u2014@NBA teams
+\u2014@MLB teams
+\u2014@NHL teams
+
+...and still have $80 billion left. $AAPL"
+Are you in Sync with the market? https://t.co/ZtHHCrSAf8 #stocks #finance #investing #trading $AAPL $LNKD $NFLX  $GOOGL $FB
+The Last Time These Insiders Purchased This Stock It Sky Rocketed 1000%+ https://t.co/bmNAHBoQBD $DIA $QQQ $SPY $GOOG $AAPL $BAC $TWTR $FB
+"This Hacker Made Amazon\u2019s Alexa, Google Now, and Siri Command One Another
+https://t.co/YXP3yqmf4H $AAPL $AMZN $GOOG https://t.co/NG7r6qgfRt"
+"Over the last 3 years, the top 14 automakers upped their combined R&amp;D spend by $192 million.
+
+$AAPL upped R&amp;D spend by $5 billion.
+
+- MS"
+Volatility can be your friend. https://t.co/aHz2r8HHD2 #stocks #trading #investing #financials #learntodaytrade $FB $AAPL $LNKD $NFLX
+"PERCENTAGE of Apple's Revenues:
+FY 2006:
+iPod 40%
+Mac 38%
+Services 10%
+Others 12%
+
+FY 2015:
+iPhone 66%
+Mac 11%
+iPad 10%
+Others 13%
+
+$AAPL"
+Apple recovered $40 million worth of gold from recycled iPhones, iPads &amp; Macs in 2015. https://t.co/XPBWlM6cBs $AAPL https://t.co/P0LMSRw7Ot
+"Apple's iPhone sales sink for 1st time ever last quarter
+https://t.co/TAKjUwl4Yc  @DavidGoldmanCNN @cnntech  $AAPL https://t.co/OrDp4BDpsD"
+$BAC is down 5% since our article was posted on Friday https://t.co/al8AgaSsiI $DIA $QQQ $SPY $AAPL $GOOG $FB $TWTR $STUDY $NFLX $LNKD $IBM
+Ben Franklin: The First Proponent Of Dividend Growth Investing? https://t.co/dx7FE2G9AH $AAPL $ACN $AL $BEN $CSV $HON $IJR $JNJ $JWN $PEGI
+$5,000 Friday the 13th - Yesterday's Futures Trades Pay Off Nicely $USO $SPY $AAPL  -- https://t.co/3RUEjAq1bO https://t.co/2L7cdebTlT
+I DON'T SEE ANY BUBBLE RIGHT NOW , I SWEAR ! $SPX $SPY $DIA $DJI $AAPL  $VIX $TVIX $C $BAC $GM $GE $FB #STOCKMARKET https://t.co/E5954RIpC7
+Terrible $AAPL quarter, finally. On the way to becoming $NOK. Tech is mean reverting, today's leaders are almost always tomorrow's laggards.
+The iPhone 7S could look radically different from the iPhones of today https://t.co/eQxUMAZ4eM $AAPL https://t.co/HIH3QqKpIC
+"No Bull: The Evidence
+https://t.co/Md2SNpjdwd
+$SPX $MSFT $GOOGL $AAPL $NFLX $AMZN $FB $DIS $V $BAC $GS $WMT $SBUX https://t.co/1oISHNX4cJ"
+The iPhone 7S could look radically different from the iPhones of today https://t.co/KgeVSjmcGe $AAPL https://t.co/7hFtg37oJu
+There was a 3rd Apple founder, Ronald Wayne, who sold his 10% stake for $800 in 1976. Today his share would've been worth $65 Billion. $AAPL
+Twitter Stock Set to Breakout Soon https://t.co/u4V6ChhpOW $TWTR $DIA $QQQ $SPY $AAPL $GLD $GDX $NUGT $DUST $BAC $GOOG $FB $STUDY $NFLX $IBM
+Alibaba Stock Price Breaks The 50 Day Moving Average https://t.co/ABOVWI6j2G $BABA $AAPL $YHOO $COST $UWTI $CSC $MON https://t.co/VlWGDxrQXh
+I still can\u2019t shake the feeling that $AAPL is slowly taking themselves private. https://t.co/XIAMvppDWh https://t.co/kdMGCGbMaJ
+$SPX ROADMAP 2016 #STOCKMARKET $INTC $F $SPY $AAPL $AMZN $C $VIX $FB $TWTR $GOOGL $UVXY $FAZ $FEZ $MSFT $GS $BAC $AA https://t.co/owuQ9awcDw
+"Want to know why $GOOG is so impressive and why $AAPL is so fucked? Read this years founders' letter from $GOOG:
+
+https://t.co/LiBjGZwyKw"
+"GET READY. Here are the companies reporting earnings next week: https://t.co/NXptPkQX70
+
+$AAPL $FB $TWTR $CMG $GILD https://t.co/tcIoCZdOZi"
+$SPX THIS TIME IT'S DIFFERENT! $SPY $DIA $SDOW $S  $FAZ $FEZ $AAPL $MSFT $BAC $C $JPM $GS $SIRI $AMZN $F $VIX $TVIX https://t.co/pkYVgNKv3P
+$SPX ROADMAP 2016 #STOCKS $TVIX $VXX $VALE $AAPL $AKS $FCX $MSFT $AA $MU $VIX $SPX $SPY #TRADING $PCLN $SIRI $ MCD https://t.co/6UH5He38h1
+The iPhone 6S is the first iPhone ever to sell fewer models than its predecessor https://t.co/s8iQOvPQeR $AAPL https://t.co/QmQROtQ9vY
+11/ For example, buy an Echo and see your behavior change. The future is happening, and $AAPL seems, to me, asleep.
+$RLYP $SPY $KORS $WDAY $MSFT $AAPL $QLIK $TIVO $NXPI $CPXX $AVGO $ZOES $LE $TICC $SLB $FCEL $VRA $MLNX $ASNA $ICPT https://t.co/LXMpz4rFG0
+#STOCKMARKET GRAVITY LESSONS: what goes up  must come down $SPX $SPY $DIA $QQQ $TVIX $VIX $AAPL $C $FB $PCLN $BAC $F https://t.co/8HQHBEgSj5
+Should Icahn's exit or Buffett's entry affect your $AAPL judgment? The Big Name Effect. https://t.co/9Z2ok61MUh https://t.co/udAQLfdJFe
+Apple revenue drops 13 percent, ending 13 years of growth. Greater China was especially weak, down 26 percent. $AAPL https://t.co/q4ovXUenBU
+It was a $18 billion day for Apple. https://t.co/iRbGeoTmCJ $AAPL
+"Apple has $233 billion in cash. It could buy:
+
+Uber
+Tesla
+Twitter
+Airbnb
+Netflix
+Yahoo
+
+...and still have $18 billion left. $AAPL"
+#3 TOP 2111.05 #STOCKS #STOCKMARKET #TRADING $SPX $SPY $VIX $TVIX $AAPL $SIRI $C $BAC $JPM $AMZN $MSFT $FB $TWTR $F https://t.co/gSqmN0fVON
+Google #IO16: Android's failure to innovate hands a Apple free run at WWDC $GOOG $AAPL https://t.co/FTs9M8JD5g https://t.co/20uou1gUkW
+$SPX 2134.72..2116.48...2111.05 HOUSTON WE HAVE A PROBLEM ! #STOCKMARKET $VIX $SPY $DIA $AAPL $C $BAC $FB $VXX $MSFT https://t.co/du3QfPUM4Q
+top #earnings $FB $AAPL $AMZN $TWTR $CMG $F $GILD $LNKD $FCX $CELG $SWKS $JBLU $T $NXPI $BA  https://t.co/lObOE0uRjZ https://t.co/94F6GJc3hE
+The iPhone 6S is the first iPhone ever to sell fewer models than its predecessor https://t.co/ZVeQ9a4Yrh $AAPL https://t.co/2Ntpbxwlyo
+You do not want to miss this incredibly candid look into $AAPL w/ @tim_cook! Tune into @MadMoneyOnCNBC on @CNBC now! https://t.co/budv4qfvju
+Foxconn axes 60,000 jobs in one Chinese factory as robots take over: https://t.co/BnFdjGCmLf $AAPL https://t.co/WhRHer8jdN
+Warren Buffett's Berkshire Hathaway reports 9.8M share stake in $AAPL https://t.co/nXmvK6PV7M https://t.co/MAcMz0iTg6
+Apple is about to report its worst quarter in 13 years on Tuesday https://t.co/NJ3hwunHCx $AAPL https://t.co/YLTmnpqNjI
+Everyone who wants an iPhone has one. $AAPL is now a consumer staple stock and will trade on replacement / shareholder yield.
+Financial Armageddon\u2019 is imminent, the next major crash will happen in 2016 $VXX $VIX $TVIX  $SPX $SPY $AAPL $MSFT $BAC $C $FB $DJI $DIA $F
+"Apple is NO longer the largest US stock by market cap. Google is: https://t.co/i81Y83jQJC
+
+$GOOGL $AAPL https://t.co/cRCKRYBICS"
+Exclusive: Apple hires former Tesla VP Chris Porritt for \u2018special [car] project\u2019 https://t.co/7knsloxvJW $TSLA $AAPL https://t.co/X8cYztExoP
+$SPX on the top of downtrend Channel Be careful! #STOCKMARKET $SPY $AAPL $AMZN $TSLA $FB $QQQ $DIA $NFLX $PCLN $C $F https://t.co/UKZCyLYuBq
+UPDATE: Apple CEO Cook says in conference call that smartphone marker is 'currently not growing' $AAPL https://t.co/WeECmrdv1j
+In February Charlie Munger was asked why Berkshire owns $GM. The $AAPL stake isn't anymore complicated than this: https://t.co/Rwkb30OEgq
+Talking to @SquawkStreet about $AAPL &amp; more at @NYSE ! https://t.co/m05b68VLMp
+iPhone sales sour #Apple's earning: https://t.co/962fj9SWsc $AAPL https://t.co/nz9FRK6sNK
+People aren\u2019t upgrading smartphones as quickly and that is bad for Apple https://t.co/EOEJPfNR8Z \U0001f513 $AAPL
+"$NXPI $JBLU $FCX $AAPL $CMG
+$TWTR $EBAY $BWLD $PNRA $CRUS
+$FB $FSLR $UPS $CELG $AMZN
+$LNKD $BIDU $SWKS $GILD $HELE https://t.co/rQUmhHgYn0"
+People mad that Icahn sold $AAPL without giving them the head\u2019s up - How much in commissions did you pay him this year?
+Cool stat: $AAPL's $46 billion loss in market cap overnight is greater than the market cap of 391 S&amp;P 500 companies https://t.co/1ms1YZzTbP
+Apple. You've come a long way... https://t.co/WGvk8K8MYv $AAPL https://t.co/3Wo0hAwRAc
+"Someone is building the Internet's biggest list of stock market cliches and it's amazing: https://t.co/mIV169cF36
+
+$SPY $AAPL $EURUSD"
+JUST IN: Apple delays earnings release by one day, to April 26th after the bell. \u2022 $AAPL
+Apple's market value is down nearly three Twitters $AAPL $TWTR
+Trump warns of a tech bubble: https://t.co/6Ks1yTa4Zc $AAPL $FB $AMZN He's 100% right about this. https://t.co/dJgTLk5JOB
+Apple could sell its billionth iPhone in just a few months' time https://t.co/g6VYDFIE3d $AAPL https://t.co/jzucmxDYXe
+$SPX  KEEP BLOWING #STOCKMARKET #BUBBLE #STOCKS $MSFT $GS $AAPL $SPY $DIA $DJI $C $SIRI $PCLN  $BAC $JPM $VIX $TVIX https://t.co/GPFBb0uCLF
+Will Apple $AAPL fall from tree? 12-mo descending triangle. I've no interest to short it, but it will be wild ride https://t.co/AnjsIKmIHI
+Tim Cook shouldn't be doing TV w/out a new product. Looks desperate. Not a consumer-facing guy. $AAPL https://t.co/Z4UFSimTLg
+When will Apple will sell its billionth iPhone? It may be sooner than you think: https://t.co/5IaF018N1p $AAPL https://t.co/cCIgtKqWHA
+#Stockmarket downtrend continues next week $spx  $spy $vix $tvix $dji $aapl $jpm $bac $c $msft $pcln $wmt $ba https://t.co/1TTlgnKnZc
+$AAPL https://t.co/AFANPYHnoq
+40 years ago this month, Apple co-founder Ronald Wayne sold his 10% stake in $AAPL for $800. Current value: $61 billion.
+Warren Buffett's Berkshire Hathaway reports 9.8M share stake in $AAPL https://t.co/rXWwuyIooI https://t.co/TztgKCcWWy
+Apple's iBooks and iTunes Movies in China have been shut down after less than 7 months https://t.co/ZuGXZqSHma $AAPL https://t.co/1OHGC9YiUf
+Possible buy on $AAPL as it drops onto it's 9 DEMA support #1Broker #Bitcoin #Blockchain https://t.co/WWssD01joh https://t.co/jOKJyG9EaJ
+"Apple is down 7% after earnings.
+
+That's about $40 BILLION in market cap gone in 30 minutes. Poof.
+
+$AAPL: https://t.co/ggfmPjJjkW"
+B4 CRASH 2008 - Paulson's speech:" OUR FINANCIAL SYSTEM IS STRONG" $VXX $VIX $TVIX $UVXY $SPX $SPY $AAPL $MSFT $BAC $C $FB $DJI $DIA $F
+$ONCI is ready to RUN this week! #stockmarket #pennystocks #parabolic $CDNL $MGT $GOOGL $AAPL $TSLA $TWTR $ONCI https://t.co/wwqf0RNOix
+Apple could sell its billionth iPhone in just a few months' time https://t.co/u2qFZ440dH $AAPL https://t.co/8cAchiZ0vC
+The iPhone might radically change in 2017 $AAPL https://t.co/IXLdCfEdus https://t.co/GpdMvFZPjE
+"The growth of smartphones. On one graph.
+
+A great share via: https://t.co/2hAJlarjSM
+
+$AAPL $GOOGL $MSFT https://t.co/BAwQRvYzou"
+"$AAPL finished last quarter with $232 billion in cash, meanwhile Kanye running up debts making records for Tidal.
+
+Bro."
+Which is bullish for $AAPL if you know anything about $GS https://t.co/WWssD01joh  https://t.co/CQk8iKMI7w
+"The tech stocks with the MOST revenue
+
+1. $AAPL
+2. $AMZN
+3. $MSFT
+
+Visual by @OphirGottlieb: https://t.co/GpZ5ct2z5r https://t.co/H6sNKdtBHd"
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/highlevelapi/src/test/resources/wordcount/word.txt
----------------------------------------------------------------------
diff --git a/demos/highlevelapi/src/test/resources/wordcount/word.txt b/demos/highlevelapi/src/test/resources/wordcount/word.txt
new file mode 100644
index 0000000..edd0f51
--- /dev/null
+++ b/demos/highlevelapi/src/test/resources/wordcount/word.txt
@@ -0,0 +1,8 @@
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+word1 word2 word7 word9 word2 word3 word2 word4 word7 word3 word9 word9 word5 word5 word4 word2 word1 error
+bye
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/demos/pom.xml
----------------------------------------------------------------------
diff --git a/demos/pom.xml b/demos/pom.xml
index e9f2daf..3528e7a2 100644
--- a/demos/pom.xml
+++ b/demos/pom.xml
@@ -174,6 +174,7 @@
       <id>all-modules</id>
       <modules>
         <module>distributedistinct</module>
+        <module>highlevelapi</module>
       </modules>
     </profile>
   </profiles>
@@ -192,7 +193,6 @@
     <module>r</module>
     <module>echoserver</module>
     <module>iteration</module>
-    <module>highlevelapi</module>
   </modules>
 
   <dependencies>
@@ -232,17 +232,6 @@
         </exclusion>
       </exclusions>
     </dependency>
-    <dependency>
-      <groupId>org.apache.apex</groupId>
-      <artifactId>malhar-stream</artifactId>
-      <version>${project.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>*</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Average.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Average.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Average.java
new file mode 100644
index 0000000..57db6d7
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Average.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.commons.lang3.tuple.MutablePair;
+
+/**
+ * Average Accumulation
+ */
+public class Average implements Accumulation<Double, MutablePair<Double, Long>, Double>
+{
+  @Override
+  public MutablePair<Double, Long> defaultAccumulatedValue()
+  {
+    return new MutablePair<>(0.0, 0L);
+  }
+  
+  @Override
+  public MutablePair<Double, Long> accumulate(MutablePair<Double, Long> accu, Double input)
+  {
+    accu.setLeft(accu.getLeft() * ((double)accu.getRight() / (accu.getRight() + 1)) + input / (accu.getRight() + 1));
+    accu.setRight(accu.getRight() + 1);
+    return accu;
+  }
+  
+  @Override
+  public MutablePair<Double, Long> merge(MutablePair<Double, Long> accu1, MutablePair<Double, Long> accu2)
+  {
+    accu1.setLeft(accu1.getLeft() * ((double)accu1.getRight() / accu1.getRight() + accu2.getRight()) +
+        accu2.getLeft() * ((double)accu2.getRight() / accu1.getRight() + accu2.getRight()));
+    accu1.setRight(accu1.getRight() + accu2.getRight());
+    return accu1;
+  }
+  
+  @Override
+  public Double getOutput(MutablePair<Double, Long> accumulatedValue)
+  {
+    return accumulatedValue.getLeft();
+  }
+  
+  @Override
+  public Double getRetraction(Double value)
+  {
+    // TODO: Need to add implementation for retraction.
+    return 0.0;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Count.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Count.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Count.java
new file mode 100644
index 0000000..2c01a0b
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Count.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.commons.lang3.mutable.MutableLong;
+
+/**
+ * Count Accumulation
+ */
+public class Count implements Accumulation<Long, MutableLong, Long>
+{
+
+  @Override
+  public MutableLong defaultAccumulatedValue()
+  {
+    return new MutableLong(0);
+  }
+
+  @Override
+  public MutableLong accumulate(MutableLong accumulatedValue, Long input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+
+  @Override
+  public MutableLong merge(MutableLong accumulatedValue1, MutableLong accumulatedValue2)
+  {
+    accumulatedValue1.add(accumulatedValue2);
+    return accumulatedValue1;
+  }
+
+  @Override
+  public Long getOutput(MutableLong accumulatedValue)
+  {
+    return accumulatedValue.getValue();
+  }
+
+  @Override
+  public Long getRetraction(Long value)
+  {
+    return -value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFn.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFn.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFn.java
new file mode 100644
index 0000000..5716cad
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFn.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * Fold Accumulation Adaptor class
+ */
+public abstract class FoldFn<INPUT, OUTPUT> implements Accumulation<INPUT, OUTPUT, OUTPUT>
+{
+
+  public FoldFn()
+  {
+  }
+
+  public FoldFn(OUTPUT initialVal)
+  {
+    this.initialVal = initialVal;
+  }
+
+  private OUTPUT initialVal;
+
+  @Override
+  public OUTPUT defaultAccumulatedValue()
+  {
+    return initialVal;
+  }
+
+  @Override
+  public OUTPUT accumulate(OUTPUT accumulatedValue, INPUT input)
+  {
+    return fold(accumulatedValue, input);
+  }
+
+  @Override
+  public OUTPUT getOutput(OUTPUT accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+
+  @Override
+  public OUTPUT getRetraction(OUTPUT value)
+  {
+    return null;
+  }
+
+  abstract OUTPUT fold(OUTPUT result, INPUT input);
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Group.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Group.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Group.java
new file mode 100644
index 0000000..632cad5
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Group.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * Group accumulation.
+ */
+public class Group<T> implements Accumulation<T, List<T>, List<T>>
+{
+  @Override
+  public List<T> defaultAccumulatedValue()
+  {
+    return new ArrayList<>();
+  }
+  
+  @Override
+  public List<T> accumulate(List<T> accumulatedValue, T input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+  
+  @Override
+  public List<T> merge(List<T> accumulatedValue1, List<T> accumulatedValue2)
+  {
+    accumulatedValue1.addAll(accumulatedValue2);
+    return accumulatedValue1;
+  }
+  
+  @Override
+  public List<T> getOutput(List<T> accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+  
+  @Override
+  public List<T> getRetraction(List<T> value)
+  {
+    // TODO: Need to add implementation for retraction.
+    return new ArrayList<>();
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Max.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Max.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Max.java
new file mode 100644
index 0000000..1002b49
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Max.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.Comparator;
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * Max accumulation.
+ */
+public class Max<T> implements Accumulation<T, T, T>
+{
+  
+  Comparator<T> comparator;
+  
+  public void setComparator(Comparator<T> comparator)
+  {
+    this.comparator = comparator;
+  }
+  
+  @Override
+  public T defaultAccumulatedValue()
+  {
+    return null;
+  }
+  
+  @Override
+  public T accumulate(T accumulatedValue, T input)
+  {
+    if (accumulatedValue == null) {
+      return input;
+    } else if (comparator != null) {
+      return (comparator.compare(input, accumulatedValue) > 0) ? input : accumulatedValue;
+    } else if (input instanceof Comparable) {
+      return (((Comparable)input).compareTo(accumulatedValue) > 0) ? input : accumulatedValue;
+    } else {
+      throw new RuntimeException("Tuple cannot be compared");
+    }
+  }
+  
+  @Override
+  public T merge(T accumulatedValue1, T accumulatedValue2)
+  {
+    return accumulate(accumulatedValue1, accumulatedValue2);
+  }
+  
+  @Override
+  public T getOutput(T accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+  
+  @Override
+  public T getRetraction(T value)
+  {
+    // TODO: Need to add implementation for retraction.
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Min.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Min.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Min.java
new file mode 100644
index 0000000..66248f4
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/Min.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.Comparator;
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * Min accumulation
+ */
+public class Min<T> implements Accumulation<T, T, T>
+{
+  
+  Comparator<T> comparator;
+  
+  public void setComparator(Comparator<T> comparator)
+  {
+    this.comparator = comparator;
+  }
+  
+  @Override
+  public T defaultAccumulatedValue()
+  {
+    return null;
+  }
+  
+  @Override
+  public T accumulate(T accumulatedValue, T input)
+  {
+    if (accumulatedValue == null) {
+      return input;
+    } else if (comparator != null) {
+      return (comparator.compare(input, accumulatedValue) < 0) ? input : accumulatedValue;
+    } else if (input instanceof Comparable) {
+      return (((Comparable)input).compareTo(accumulatedValue) < 0) ? input : accumulatedValue;
+    } else {
+      throw new RuntimeException("Tuple cannot be compared");
+    }
+  }
+  
+  @Override
+  public T merge(T accumulatedValue1, T accumulatedValue2)
+  {
+    return accumulate(accumulatedValue1, accumulatedValue2);
+  }
+  
+  @Override
+  public T getOutput(T accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+  
+  @Override
+  public T getRetraction(T value)
+  {
+    // TODO: Need to add implementation for retraction.
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFn.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFn.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFn.java
new file mode 100644
index 0000000..c21ab32
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFn.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * An easy to use reduce Accumulation
+ * @param <INPUT>
+ */
+public abstract class ReduceFn<INPUT> implements Accumulation<INPUT, INPUT, INPUT>
+{
+  @Override
+  public INPUT defaultAccumulatedValue()
+  {
+    return null;
+  }
+
+  @Override
+  public INPUT accumulate(INPUT accumulatedValue, INPUT input)
+  {
+    if (accumulatedValue == null) {
+      return input;
+    }
+    return reduce(accumulatedValue, input);
+  }
+
+  @Override
+  public INPUT merge(INPUT accumulatedValue1, INPUT accumulatedValue2)
+  {
+    return reduce(accumulatedValue1, accumulatedValue2);
+  }
+
+  @Override
+  public INPUT getOutput(INPUT accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+
+  @Override
+  public INPUT getRetraction(INPUT value)
+  {
+    return null;
+  }
+
+  public abstract INPUT reduce(INPUT input1, INPUT input2);
+
+
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicates.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicates.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicates.java
new file mode 100644
index 0000000..b7cd770
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicates.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * RemoveDuplicates Accumulation.
+ * @param <T>
+ */
+public class RemoveDuplicates<T> implements Accumulation<T, Set<T>, List<T>>
+{
+  @Override
+  public Set<T> defaultAccumulatedValue()
+  {
+    return new HashSet<>();
+  }
+  
+  @Override
+  public Set<T> accumulate(Set<T> accumulatedValue, T input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+  
+  @Override
+  public Set<T> merge(Set<T> accumulatedValue1, Set<T> accumulatedValue2)
+  {
+    for (T item : accumulatedValue2) {
+      accumulatedValue1.add(item);
+    }
+    return accumulatedValue1;
+  }
+  
+  @Override
+  public List<T> getOutput(Set<T> accumulatedValue)
+  {
+    if (accumulatedValue == null) {
+      return new ArrayList<>();
+    } else {
+      return new ArrayList<>(accumulatedValue);
+    }
+  }
+  
+  @Override
+  public List<T> getRetraction(List<T> value)
+  {
+    // TODO: Need to add implementation for retraction.
+    return new ArrayList<>(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumDouble.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumDouble.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumDouble.java
new file mode 100644
index 0000000..60b195b
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumDouble.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.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.commons.lang.mutable.MutableDouble;
+
+/**
+ * Sum Accumulation for doubles.
+ */
+public class SumDouble implements Accumulation<Double, MutableDouble, Double>
+{
+  @Override
+  public MutableDouble defaultAccumulatedValue()
+  {
+    return new MutableDouble(0.0);
+  }
+  
+  @Override
+  public MutableDouble accumulate(MutableDouble accumulatedValue, Double input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+  
+  @Override
+  public MutableDouble merge(MutableDouble accumulatedValue1, MutableDouble accumulatedValue2)
+  {
+    accumulatedValue1.add(accumulatedValue2);
+    return accumulatedValue1;
+  }
+  
+  @Override
+  public Double getOutput(MutableDouble accumulatedValue)
+  {
+    return accumulatedValue.doubleValue();
+  }
+  
+  @Override
+  public Double getRetraction(Double value)
+  {
+    return -value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumFloat.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumFloat.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumFloat.java
new file mode 100644
index 0000000..14e69e2
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumFloat.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.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.commons.lang.mutable.MutableFloat;
+
+/**
+ * Sum Accumulation for floats.
+ */
+public class SumFloat implements Accumulation<Float, MutableFloat, Float>
+{
+  @Override
+  public MutableFloat defaultAccumulatedValue()
+  {
+    return new MutableFloat(0.);
+  }
+  
+  @Override
+  public MutableFloat accumulate(MutableFloat accumulatedValue, Float input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+  
+  @Override
+  public MutableFloat merge(MutableFloat accumulatedValue1, MutableFloat accumulatedValue2)
+  {
+    accumulatedValue1.add(accumulatedValue2);
+    return accumulatedValue1;
+  }
+  
+  @Override
+  public Float getOutput(MutableFloat accumulatedValue)
+  {
+    return accumulatedValue.floatValue();
+  }
+  
+  @Override
+  public Float getRetraction(Float value)
+  {
+    return -value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumInt.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumInt.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumInt.java
new file mode 100644
index 0000000..886a7d0
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumInt.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.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.commons.lang.mutable.MutableInt;
+
+/**
+ * Sum accumulation for integers.
+ */
+public class SumInt implements Accumulation<Integer, MutableInt, Integer>
+{
+  @Override
+  public MutableInt defaultAccumulatedValue()
+  {
+    return new MutableInt(0);
+  }
+  
+  @Override
+  public MutableInt accumulate(MutableInt accumulatedValue, Integer input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+  
+  @Override
+  public MutableInt merge(MutableInt accumulatedValue1, MutableInt accumulatedValue2)
+  {
+    accumulatedValue1.add(accumulatedValue2);
+    return accumulatedValue1;
+  }
+  
+  @Override
+  public Integer getOutput(MutableInt accumulatedValue)
+  {
+    return accumulatedValue.intValue();
+  }
+  
+  @Override
+  public Integer getRetraction(Integer value)
+  {
+    return -value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumLong.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumLong.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumLong.java
new file mode 100644
index 0000000..469eef9
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumLong.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.apex.malhar.lib.window.impl.accumulation;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import org.apache.commons.lang.mutable.MutableLong;
+
+/**
+ * Sum accumulation for longs.
+ */
+public class SumLong implements Accumulation<Long, MutableLong, Long>
+{
+  @Override
+  public MutableLong defaultAccumulatedValue()
+  {
+    return new MutableLong(0L);
+  }
+  
+  @Override
+  public MutableLong accumulate(MutableLong accumulatedValue, Long input)
+  {
+    accumulatedValue.add(input);
+    return accumulatedValue;
+  }
+  
+  @Override
+  public MutableLong merge(MutableLong accumulatedValue1, MutableLong accumulatedValue2)
+  {
+    accumulatedValue1.add(accumulatedValue2);
+    return accumulatedValue1;
+  }
+  
+  @Override
+  public Long getOutput(MutableLong accumulatedValue)
+  {
+    return accumulatedValue.longValue();
+  }
+  
+  @Override
+  public Long getRetraction(Long value)
+  {
+    return -value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopN.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopN.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopN.java
new file mode 100644
index 0000000..7dad8cc
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopN.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+
+/**
+ * TopN accumulation
+ */
+public class TopN<T> implements Accumulation<T, List<T>, List<T>>
+{
+  int n;
+
+  Comparator<T> comparator;
+
+  public void setN(int n)
+  {
+    this.n = n;
+  }
+
+  public void setComparator(Comparator<T> comparator)
+  {
+    this.comparator = comparator;
+  }
+
+  @Override
+  public List<T> defaultAccumulatedValue()
+  {
+    return new LinkedList<>();
+  }
+
+  @Override
+  public List<T> accumulate(List<T> accumulatedValue, T input)
+  {
+    int k = 0;
+    for (T inMemory : accumulatedValue) {
+      if (comparator != null) {
+        if (comparator.compare(inMemory, input) < 0) {
+          break;
+        }
+      } else if (input instanceof Comparable) {
+        if (((Comparable<T>)input).compareTo(inMemory) > 0) {
+          break;
+        }
+      } else {
+        throw new RuntimeException("Tuple cannot be compared");
+      }
+      k++;
+    }
+    accumulatedValue.add(k, input);
+    if (accumulatedValue.size() > n) {
+      accumulatedValue.remove(accumulatedValue.get(accumulatedValue.size() - 1));
+    }
+    return accumulatedValue;
+  }
+
+  @Override
+  public List<T> merge(List<T> accumulatedValue1, List<T> accumulatedValue2)
+  {
+    accumulatedValue1.addAll(accumulatedValue2);
+    if (comparator != null) {
+      Collections.sort(accumulatedValue1, Collections.reverseOrder(comparator));
+    } else {
+      Collections.sort(accumulatedValue1, Collections.reverseOrder());
+    }
+    if (accumulatedValue1.size() > n) {
+      return accumulatedValue1.subList(0, n);
+    } else {
+      return accumulatedValue1;
+    }
+  }
+
+  @Override
+  public List<T> getOutput(List<T> accumulatedValue)
+  {
+    return accumulatedValue;
+  }
+
+  @Override
+  public List<T> getRetraction(List<T> accumulatedValue)
+  {
+    return new LinkedList<>();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKey.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKey.java b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKey.java
new file mode 100644
index 0000000..d9f9cfd
--- /dev/null
+++ b/library/src/main/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKey.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.apex.malhar.lib.window.Accumulation;
+import com.datatorrent.lib.util.KeyValPair;
+
+/**
+ * Generalized TopNByKey accumulation
+ */
+public class TopNByKey<K, V> implements
+    Accumulation<KeyValPair<K, V>, Map<K, V>, List<KeyValPair<K, V>>>
+{
+  int n = 10;
+
+  Comparator<V> comparator;
+
+  public void setN(int n)
+  {
+    this.n = n;
+  }
+
+  public void setComparator(Comparator<V> comparator)
+  {
+    this.comparator = comparator;
+  }
+
+  @Override
+  public Map<K, V> defaultAccumulatedValue()
+  {
+    return new HashMap<>();
+  }
+
+  @Override
+  public Map<K, V> accumulate(Map<K, V> accumulatedValue, KeyValPair<K, V> input)
+  {
+    accumulatedValue.put(input.getKey(), input.getValue());
+    return accumulatedValue;
+  }
+
+  @Override
+  public Map<K, V> merge(Map<K, V> accumulatedValue1, Map<K, V> accumulatedValue2)
+  {
+    for (Map.Entry<K, V> entry : accumulatedValue2.entrySet()) {
+      if (!accumulatedValue1.containsKey(entry.getKey())) {
+        accumulatedValue1.put(entry.getKey(), entry.getValue());
+      } else if (comparator != null) {
+        if (comparator.compare(entry.getValue(), accumulatedValue1.get(entry.getKey())) > 0) {
+          accumulatedValue1.put(entry.getKey(), entry.getValue());
+        }
+      } else if (entry.getValue() instanceof Comparable) {
+        if (((Comparable<V>)entry.getValue()).compareTo(accumulatedValue1.get(entry.getKey())) > 0) {
+          accumulatedValue1.put(entry.getKey(), entry.getValue());
+        }
+      }
+    }
+    return accumulatedValue1;
+  }
+
+  @Override
+  public List<KeyValPair<K, V>> getOutput(Map<K, V> accumulatedValue)
+  {
+    LinkedList<KeyValPair<K, V>> result = new LinkedList<>();
+    for (Map.Entry<K, V> entry : accumulatedValue.entrySet()) {
+      int k = 0;
+      for (KeyValPair<K, V> inMemory : result) {
+        if (comparator != null) {
+          if (comparator.compare(entry.getValue(), inMemory.getValue()) > 0) {
+            break;
+          }
+        } else if (entry.getValue() instanceof Comparable) {
+          if (((Comparable<V>)entry.getValue()).compareTo(inMemory.getValue()) > 0) {
+            break;
+          }
+        }
+        k++;
+      }
+      result.add(k, new KeyValPair<K, V>(entry.getKey(), entry.getValue()));
+      if (result.size() > n) {
+        result.remove(result.get(result.size() - 1));
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public List<KeyValPair<K, V>> getRetraction(List<KeyValPair<K, V>> value)
+  {
+    // TODO: Need to add implementation for retraction.
+    return new LinkedList<>();
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/AverageTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/AverageTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/AverageTest.java
new file mode 100644
index 0000000..fb4de3c
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/AverageTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.commons.lang3.tuple.MutablePair;
+
+/**
+ * Test for {@link Average}.
+ */
+public class AverageTest
+{
+  @Test
+  public void AverageTest()
+  {
+    Average ave = new Average();
+    MutablePair<Double, Long> accu = ave.defaultAccumulatedValue();
+    
+    for (int i = 1; i <= 10; i++) {
+      accu = ave.accumulate(accu, (double)i);
+    }
+    Assert.assertTrue(5.5 == accu.getLeft());
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFnTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFnTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFnTest.java
new file mode 100644
index 0000000..4e6f8f1
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/FoldFnTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.apex.malhar.lib.window.Tuple;
+import com.datatorrent.api.DefaultInputPort;
+import com.datatorrent.api.DefaultOutputPort;
+import com.datatorrent.api.InputOperator;
+import com.datatorrent.common.util.BaseOperator;
+
+/**
+ * Test for {@link ReduceFn}.
+ */
+public class FoldFnTest
+{
+  public static class NumGen extends BaseOperator implements InputOperator
+  {
+    public transient DefaultOutputPort<Integer> output = new DefaultOutputPort<>();
+  
+    public static int count = 0;
+    private int i = 0;
+  
+    public NumGen()
+    {
+      count = 0;
+      i = 0;
+    }
+  
+    @Override
+    public void emitTuples()
+    {
+      while (i <= 7) {
+        try {
+          Thread.sleep(50);
+        } catch (InterruptedException e) {
+          // Ignore it.
+        }
+        count++;
+        if (i >= 0) {
+          output.emit(i++);
+        }
+      }
+      i = -1;
+    }
+  }
+  
+  public static class Collector extends BaseOperator
+  {
+    private static int result;
+    
+    public transient DefaultInputPort<Tuple.WindowedTuple<Integer>> input = new DefaultInputPort<Tuple.WindowedTuple<Integer>>()
+    {
+      @Override
+      public void process(Tuple.WindowedTuple<Integer> tuple)
+      {
+        result = tuple.getValue();
+      }
+    };
+    
+    public int getResult()
+    {
+      return result;
+    }
+  }
+  
+  public static class Plus extends FoldFn<Integer, Integer>
+  {
+    @Override
+    public Integer merge(Integer accumulatedValue1, Integer accumulatedValue2)
+    {
+      return fold(accumulatedValue1, accumulatedValue2);
+    }
+    
+    @Override
+    public Integer fold(Integer input1, Integer input2)
+    {
+      if (input1 == null) {
+        return input2;
+      }
+      return input1 + input2;
+    }
+  }
+  
+  @Test
+  public void FoldFnTest()
+  {
+    
+    FoldFn<String, String> concat = new FoldFn<String, String>()
+    {
+      @Override
+      public String merge(String accumulatedValue1, String accumulatedValue2)
+      {
+        return fold(accumulatedValue1, accumulatedValue2);
+      }
+  
+      @Override
+      public String fold(String input1, String input2)
+      {
+        return input1 + ", " + input2;
+      }
+    };
+    
+    String[] ss = new String[]{"b", "c", "d", "e"};
+    String base = "a";
+    
+    for (String s : ss) {
+      base = concat.accumulate(base, s);
+    }
+    Assert.assertEquals("a, b, c, d, e", base);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/GroupTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/GroupTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/GroupTest.java
new file mode 100644
index 0000000..a9aac77
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/GroupTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.List;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link Group}.
+ */
+public class GroupTest
+{
+  @Test
+  public void GroupTest()
+  {
+    Group<Integer> group = new Group<>();
+    
+    List<Integer> accu = group.defaultAccumulatedValue();
+    Assert.assertEquals(0, accu.size());
+    Assert.assertEquals(1, group.accumulate(accu, 10).size());
+    Assert.assertEquals(2, group.accumulate(accu, 11).size());
+    Assert.assertEquals(3, group.accumulate(accu, 11).size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MaxTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MaxTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MaxTest.java
new file mode 100644
index 0000000..c873125
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MaxTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.Comparator;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for Max accumulation
+ */
+public class MaxTest
+{
+  @Test
+  public void MaxTest()
+  {
+    Max<Integer> max = new Max<>();
+    
+    Assert.assertEquals((Integer)5, max.accumulate(5, 3));
+    Assert.assertEquals((Integer)6, max.accumulate(4, 6));
+    Assert.assertEquals((Integer)5, max.merge(5, 2));
+  
+    Comparator<Integer> com = new Comparator<Integer>()
+    {
+      @Override
+      public int compare(Integer o1, Integer o2)
+      {
+        return -(o1.compareTo(o2));
+      }
+    };
+    
+    max.setComparator(com);
+    Assert.assertEquals((Integer)3, max.accumulate(5, 3));
+    Assert.assertEquals((Integer)4, max.accumulate(4, 6));
+    Assert.assertEquals((Integer)2, max.merge(5, 2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MinTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MinTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MinTest.java
new file mode 100644
index 0000000..74816b0
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/MinTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.Comparator;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link Min}.
+ */
+public class MinTest
+{
+  @Test
+  public void MinTest()
+  {
+    Min<Integer> min = new Min<>();
+    
+    Assert.assertEquals((Integer)3, min.accumulate(5, 3));
+    Assert.assertEquals((Integer)4, min.accumulate(4, 6));
+    Assert.assertEquals((Integer)2, min.merge(5, 2));
+    
+    Comparator<Integer> com = new Comparator<Integer>()
+    {
+      @Override
+      public int compare(Integer o1, Integer o2)
+      {
+        return -(o1.compareTo(o2));
+      }
+    };
+    
+    min.setComparator(com);
+    Assert.assertEquals((Integer)5, min.accumulate(5, 3));
+    Assert.assertEquals((Integer)6, min.accumulate(4, 6));
+    Assert.assertEquals((Integer)5, min.merge(5, 2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFnTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFnTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFnTest.java
new file mode 100644
index 0000000..6b5bbad
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/ReduceFnTest.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.apex.malhar.lib.window.impl.accumulation;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link ReduceFn}.
+ */
+public class ReduceFnTest
+{
+  
+  @Test
+  public void ReduceFnTest()
+  {
+    ReduceFn<String> concat = new ReduceFn<String>()
+    {
+      @Override
+      public String reduce(String input1, String input2)
+      {
+        return input1 + ", " + input2;
+      }
+    };
+    
+    String[] ss = new String[]{"b", "c", "d", "e"};
+    String base = "a";
+    
+    for (String s : ss) {
+      base = concat.accumulate(base, s);
+    }
+    Assert.assertEquals("a, b, c, d, e", base);
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicatesTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicatesTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicatesTest.java
new file mode 100644
index 0000000..f0196d2
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/RemoveDuplicatesTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.Set;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for {@link RemoveDuplicates}.
+ */
+public class RemoveDuplicatesTest
+{
+  @Test
+  public void RemoveDuplicatesTest()
+  {
+    RemoveDuplicates<Integer> rd = new RemoveDuplicates<>();
+    
+    Set<Integer> accu = rd.defaultAccumulatedValue();
+    Assert.assertEquals(0, accu.size());
+    Assert.assertEquals(1, rd.accumulate(accu, 10).size());
+    Assert.assertEquals(2, rd.accumulate(accu, 11).size());
+    Assert.assertEquals(2, rd.accumulate(accu, 11).size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumTest.java
new file mode 100644
index 0000000..65b6480
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/SumTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.commons.lang.mutable.MutableDouble;
+import org.apache.commons.lang.mutable.MutableFloat;
+import org.apache.commons.lang.mutable.MutableInt;
+import org.apache.commons.lang.mutable.MutableLong;
+
+/**
+ * Test for different Sum Accumulations.
+ */
+public class SumTest
+{
+  @Test
+  public void SumTest()
+  {
+    SumInt si = new SumInt();
+    SumLong sl = new SumLong();
+    SumFloat sf = new SumFloat();
+    SumDouble sd = new SumDouble();
+    
+    Assert.assertEquals(new MutableInt(10), si.accumulate(si.defaultAccumulatedValue(), 10));
+    Assert.assertEquals(new MutableInt(11), si.accumulate(new MutableInt(1), 10));
+    Assert.assertEquals(new MutableInt(22), si.merge(new MutableInt(1), new MutableInt(21)));
+    
+    Assert.assertEquals(new MutableLong(10L), sl.accumulate(sl.defaultAccumulatedValue(), 10L));
+    Assert.assertEquals(new MutableLong(22L), sl.accumulate(new MutableLong(2L), 20L));
+    Assert.assertEquals(new MutableLong(41L), sl.merge(new MutableLong(32L), new MutableLong(9L)));
+    
+    Assert.assertEquals(new MutableFloat(9.0F), sf.accumulate(sf.defaultAccumulatedValue(), 9.0F));
+    Assert.assertEquals(new MutableFloat(22.5F), sf.accumulate(new MutableFloat(2.5F), 20F));
+    Assert.assertEquals(new MutableFloat(41.0F), sf.merge(new MutableFloat(33.1F), new MutableFloat(7.9F)));
+    
+    Assert.assertEquals(new MutableDouble(9.0), sd.accumulate(sd.defaultAccumulatedValue(), 9.0));
+    Assert.assertEquals(new MutableDouble(22.5), sd.accumulate(new MutableDouble(2.5), 20.0));
+    Assert.assertEquals(new MutableDouble(41.0), sd.merge(new MutableDouble(33.1), new MutableDouble(7.9)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKeyTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKeyTest.java b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKeyTest.java
new file mode 100644
index 0000000..3f6ac09
--- /dev/null
+++ b/library/src/test/java/org/apache/apex/malhar/lib/window/impl/accumulation/TopNByKeyTest.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.apex.malhar.lib.window.impl.accumulation;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.datatorrent.lib.util.KeyValPair;
+
+/**
+ * Unit test for TopNByKey accumulation
+ */
+public class TopNByKeyTest
+{
+  @Test
+  public void TopNByKeyTest() throws Exception
+  {
+    TopNByKey<String, Integer> topNByKey = new TopNByKey<>();
+    topNByKey.setN(3);
+    Map<String, Integer> accu = topNByKey.defaultAccumulatedValue();
+  
+    Assert.assertEquals(0, accu.size());
+    
+    accu = topNByKey.accumulate(accu, new KeyValPair<String, Integer>("1", 1));
+    accu = topNByKey.accumulate(accu, new KeyValPair<String, Integer>("3", 3));
+    
+    List<KeyValPair<String, Integer>> result1 = new ArrayList<>();
+  
+    result1.add(new KeyValPair<String, Integer>("3", 3));
+    result1.add(new KeyValPair<String, Integer>("1", 1));
+    
+    Assert.assertEquals(result1, topNByKey.getOutput(accu));
+    
+    accu = topNByKey.accumulate(accu, new KeyValPair<String, Integer>("2", 2));
+  
+    List<KeyValPair<String, Integer>> result2 = new ArrayList<>();
+  
+    result2.add(new KeyValPair<String, Integer>("3", 3));
+    result2.add(new KeyValPair<String, Integer>("2", 2));
+    result2.add(new KeyValPair<String, Integer>("1", 1));
+    
+    Assert.assertEquals(result2, topNByKey.getOutput(accu));
+    
+    accu = topNByKey.accumulate(accu, new KeyValPair<String, Integer>("5", 5));
+    accu = topNByKey.accumulate(accu, new KeyValPair<String, Integer>("4", 4));
+  
+    List<KeyValPair<String, Integer>> result3 = new ArrayList<>();
+    
+    result3.add(new KeyValPair<String, Integer>("5", 5));
+    result3.add(new KeyValPair<String, Integer>("4", 4));
+    result3.add(new KeyValPair<String, Integer>("3", 3));
+    
+    Assert.assertEquals(result3, topNByKey.getOutput(accu));
+  }
+}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
index bc99035..84f05fc 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/WindowedStream.java
@@ -27,9 +27,9 @@ import org.apache.apex.malhar.lib.window.TriggerOption;
 import org.apache.apex.malhar.lib.window.Tuple;
 import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl;
 import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl;
+import org.apache.apex.malhar.lib.window.impl.accumulation.FoldFn;
+import org.apache.apex.malhar.lib.window.impl.accumulation.ReduceFn;
 import org.apache.apex.malhar.stream.api.function.Function;
-import org.apache.apex.malhar.stream.api.impl.accumulation.FoldFn;
-import org.apache.apex.malhar.stream.api.impl.accumulation.ReduceFn;
 import org.apache.hadoop.classification.InterfaceStability;
 
 import com.datatorrent.lib.util.KeyValPair;

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java
index a293ea8..ebd5eea 100644
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java
+++ b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/ApexWindowedStreamImpl.java
@@ -32,15 +32,16 @@ import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedKeyedStorage;
 import org.apache.apex.malhar.lib.window.impl.InMemoryWindowedStorage;
 import org.apache.apex.malhar.lib.window.impl.KeyedWindowedOperatorImpl;
 import org.apache.apex.malhar.lib.window.impl.WindowedOperatorImpl;
+import org.apache.apex.malhar.lib.window.impl.accumulation.Count;
+import org.apache.apex.malhar.lib.window.impl.accumulation.FoldFn;
+import org.apache.apex.malhar.lib.window.impl.accumulation.ReduceFn;
+import org.apache.apex.malhar.lib.window.impl.accumulation.TopN;
+
 import org.apache.apex.malhar.stream.api.ApexStream;
 import org.apache.apex.malhar.stream.api.Option;
 import org.apache.apex.malhar.stream.api.WindowedStream;
 import org.apache.apex.malhar.stream.api.function.Function;
 
-import org.apache.apex.malhar.stream.api.impl.accumulation.Count;
-import org.apache.apex.malhar.stream.api.impl.accumulation.FoldFn;
-import org.apache.apex.malhar.stream.api.impl.accumulation.ReduceFn;
-import org.apache.apex.malhar.stream.api.impl.accumulation.TopN;
 import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.hadoop.classification.InterfaceStability;
 
@@ -62,7 +63,7 @@ public class ApexWindowedStreamImpl<T> extends ApexStreamImpl<T> implements Wind
 
   protected Duration allowedLateness;
 
-  private class ConvertFn<T> implements Function.MapFunction<T, Tuple<T>>
+  private static class ConvertFn<T> implements Function.MapFunction<T, Tuple<T>>
   {
 
     @Override

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java
deleted file mode 100644
index 68f1b9e..0000000
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/Count.java
+++ /dev/null
@@ -1,61 +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.apex.malhar.stream.api.impl.accumulation;
-
-import org.apache.apex.malhar.lib.window.Accumulation;
-import org.apache.commons.lang3.mutable.MutableLong;
-
-/**
- * Count Accumulation
- */
-public class Count implements Accumulation<Long, MutableLong, Long>
-{
-
-  @Override
-  public MutableLong defaultAccumulatedValue()
-  {
-    return new MutableLong(0);
-  }
-
-  @Override
-  public MutableLong accumulate(MutableLong accumulatedValue, Long input)
-  {
-    accumulatedValue.add(input);
-    return accumulatedValue;
-  }
-
-  @Override
-  public MutableLong merge(MutableLong accumulatedValue1, MutableLong accumulatedValue2)
-  {
-    accumulatedValue1.add(accumulatedValue2);
-    return accumulatedValue1;
-  }
-
-  @Override
-  public Long getOutput(MutableLong accumulatedValue)
-  {
-    return accumulatedValue.getValue();
-  }
-
-  @Override
-  public Long getRetraction(Long value)
-  {
-    return -value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java
deleted file mode 100644
index 3ab6892..0000000
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/FoldFn.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.apex.malhar.stream.api.impl.accumulation;
-
-import org.apache.apex.malhar.lib.window.Accumulation;
-
-/**
- * Fold Accumulation Adaptor class
- */
-public abstract class FoldFn<INPUT, OUTPUT> implements Accumulation<INPUT, OUTPUT, OUTPUT>
-{
-
-  public FoldFn()
-  {
-  }
-
-  public FoldFn(OUTPUT initialVal)
-  {
-    this.initialVal = initialVal;
-  }
-
-  private OUTPUT initialVal;
-
-  @Override
-  public OUTPUT defaultAccumulatedValue()
-  {
-    return initialVal;
-  }
-
-  @Override
-  public OUTPUT accumulate(OUTPUT accumulatedValue, INPUT input)
-  {
-    return fold(accumulatedValue, input);
-  }
-
-  @Override
-  public OUTPUT getOutput(OUTPUT accumulatedValue)
-  {
-    return accumulatedValue;
-  }
-
-  @Override
-  public OUTPUT getRetraction(OUTPUT value)
-  {
-    return null;
-  }
-
-  abstract OUTPUT fold(OUTPUT result, INPUT input);
-}

http://git-wip-us.apache.org/repos/asf/apex-malhar/blob/dcca7752/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java
----------------------------------------------------------------------
diff --git a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java b/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java
deleted file mode 100644
index b4507bc..0000000
--- a/stream/src/main/java/org/apache/apex/malhar/stream/api/impl/accumulation/ReduceFn.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.apex.malhar.stream.api.impl.accumulation;
-
-import org.apache.apex.malhar.lib.window.Accumulation;
-
-/**
- * An easy to use reduce Accumulation
- * @param <INPUT>
- */
-public abstract class ReduceFn<INPUT> implements Accumulation<INPUT, INPUT, INPUT>
-{
-  @Override
-  public INPUT defaultAccumulatedValue()
-  {
-    return null;
-  }
-
-  @Override
-  public INPUT accumulate(INPUT accumulatedValue, INPUT input)
-  {
-    if (accumulatedValue == null) {
-      return input;
-    }
-    return reduce(accumulatedValue, input);
-  }
-
-  @Override
-  public INPUT merge(INPUT accumulatedValue1, INPUT accumulatedValue2)
-  {
-    return reduce(accumulatedValue1, accumulatedValue2);
-  }
-
-  @Override
-  public INPUT getOutput(INPUT accumulatedValue)
-  {
-    return accumulatedValue;
-  }
-
-  @Override
-  public INPUT getRetraction(INPUT value)
-  {
-    return null;
-  }
-
-  public abstract INPUT reduce(INPUT input1, INPUT input2);
-
-
-}