You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2015/10/09 12:16:37 UTC

[01/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Repository: flink
Updated Branches:
  refs/heads/master 0ee0c1f55 -> 0c1141abc


http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
index 7efb006..bf63695 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
@@ -122,31 +122,31 @@ public class TopSpeedWindowingExampleData {
 					"(0,100,2005.5555555555566,1424952017667)\n" + "(1,95,2211.1111111111118,1424952017668)";
 
 	public static final String TOP_SPEEDS =
-					"(0,55,15.277777777777777,1424951918630)\n" +
+			"(0,55,15.277777777777777,1424951918630)\n" +
 					"(1,50,26.38888888888889,1424951919632)\n" +
 					"(0,65,108.33333333333333,1424951924635)\n" +
 					"(1,50,26.38888888888889,1424951919632)\n" +
 					"(0,65,108.33333333333333,1424951924635)\n" +
-					"(1,55,159.7222222222222,1424951930637)\n" +
+					"(1,65,194.4444444444444,1424951932638)\n" +
 					"(0,65,108.33333333333333,1424951924635)\n" +
 					"(1,70,213.88888888888886,1424951933638)\n" +
-					"(0,60,156.94444444444443,1424951927636)\n" +
+					"(0,60,218.05555555555551,1424951931637)\n" +
 					"(1,75,272.2222222222222,1424951936639)\n" +
+					"(0,55,233.3333333333333,1424951932637)\n" +
 					"(1,75,272.2222222222222,1424951936639)\n" +
-					"(0,60,218.05555555555551,1424951931637)\n" +
 					"(1,75,272.2222222222222,1424951936639)\n" +
 					"(0,55,288.88888888888886,1424951936639)\n" +
-					"(1,75,272.2222222222222,1424951936639)\n" +
-					"(1,75,497.2222222222222,1424951948643)\n" +
-					"(0,55,344.44444444444446,1424951940640)\n" +
+					"(1,70,329.16666666666663,1424951939640)\n" +
+					"(0,55,373.61111111111114,1424951942641)\n" +
 					"(1,80,519.4444444444443,1424951949644)\n" +
-					"(0,50,413.88888888888897,1424951945642)\n" +
 					"(1,85,586.111111111111,1424951952645)\n" +
+					"(0,50,487.50000000000006,1424951951644)\n" +
 					"(1,85,586.111111111111,1424951952645)\n" +
-					"(0,55,573.6111111111112,1424951958646)\n" +
+					"(0,60,590.2777777777778,1424951959647)\n" +
 					"(1,85,586.111111111111,1424951952645)\n" +
-					"(0,70,627.7777777777778,1424951961647)\n" +
-					"(1,90,831.9444444444441,1424951963648)\n" +
+					"(0,75,648.6111111111112,1424951962648)\n" +
+					"(1,85,715.2777777777776,1424951958647)\n" +
+					"(1,95,858.333333333333,1424951964649)\n" +
 					"(0,80,670.8333333333334,1424951963648)\n" +
 					"(1,95,858.333333333333,1424951964649)\n" +
 					"(0,80,670.8333333333334,1424951963648)\n" +
@@ -158,22 +158,22 @@ public class TopSpeedWindowingExampleData {
 					"(1,100,937.4999999999998,1424951967650)\n" +
 					"(1,100,937.4999999999998,1424951967650)\n" +
 					"(0,85,861.1111111111112,1424951972651)\n" +
-					"(1,100,965.2777777777776,1424951968650)\n" +
+					"(1,100,993.0555555555554,1424951969650)\n" +
 					"(0,85,861.1111111111112,1424951972651)\n" +
-					"(1,100,1020.8333333333333,1424951970651)\n" +
-					"(1,100,1076.388888888889,1424951972651)\n" +
-					"(0,90,1058.3333333333335,1424951981654)\n" +
+					"(1,100,1048.611111111111,1424951971651)\n" +
 					"(1,100,1130.5555555555557,1424951974652)\n" +
-					"(0,95,1133.3333333333335,1424951984655)\n" +
-					"(1,100,1186.1111111111113,1424951976653)\n" +
+					"(0,90,1058.3333333333335,1424951981654)\n" +
+					"(1,100,1158.3333333333335,1424951975652)\n" +
 					"(0,95,1133.3333333333335,1424951984655)\n" +
 					"(1,100,1240.277777777778,1424951978653)\n" +
 					"(0,95,1133.3333333333335,1424951984655)\n" +
-					"(1,100,1295.8333333333337,1424951980654)\n" +
+					"(1,100,1268.0555555555559,1424951979654)\n" +
+					"(0,95,1133.3333333333335,1424951984655)\n" +
+					"(1,100,1323.6111111111115,1424951981654)\n" +
 					"(0,95,1133.3333333333335,1424951984655)\n" +
-					"(1,100,1351.3888888888894,1424951982655)\n" +
+					"(1,100,1379.1666666666672,1424951983655)\n" +
 					"(0,100,1358.3333333333335,1424951993659)\n" +
-					"(1,100,1406.944444444445,1424951984656)\n" +
+					"(1,100,1563.8888888888896,1424951990658)\n" +
 					"(0,100,1358.3333333333335,1424951993659)\n" +
 					"(1,100,1563.8888888888896,1424951990658)\n" +
 					"(0,100,1358.3333333333335,1424951993659)\n" +
@@ -181,47 +181,47 @@ public class TopSpeedWindowingExampleData {
 					"(0,100,1358.3333333333335,1424951993659)\n" +
 					"(0,100,1358.3333333333335,1424951993659)\n" +
 					"(1,100,1669.4444444444453,1424951994659)\n" +
-					"(0,100,1386.1111111111113,1424951994659)\n" +
-					"(1,95,1695.8333333333342,1424951995660)\n" +
 					"(0,100,1440.277777777778,1424951996660)\n" +
-					"(1,90,1947.2222222222226,1424952006664)\n" +
+					"(1,90,1720.8333333333342,1424951996660)\n" +
+					"(0,100,1468.0555555555559,1424951997660)\n" +
+					"(1,95,1973.6111111111115,1424952007664)\n" +
 					"(0,100,1522.2222222222226,1424951999661)\n" +
 					"(0,100,1627.7777777777783,1424952003662)\n" +
 					"(1,95,1973.6111111111115,1424952007664)\n" +
 					"(0,100,1627.7777777777783,1424952003662)\n" +
 					"(1,95,1973.6111111111115,1424952007664)\n" +
-					"(0,100,1655.555555555556,1424952004663)\n" +
 					"(0,100,1709.7222222222229,1424952006663)\n" +
+					"(0,100,1737.5000000000007,1424952007664)\n" +
 					"(1,95,1973.6111111111115,1424952007664)\n" +
 					"(0,100,1791.6666666666674,1424952009664)\n" +
 					"(1,95,2211.1111111111118,1424952017668)\n";
 
 	public static final String TOP_CASE_CLASS_SPEEDS =
-					"CarEvent(0,55,15.277777777777777,1424951918630)\n" +
+			"CarEvent(0,55,15.277777777777777,1424951918630)\n" +
 					"CarEvent(1,50,26.38888888888889,1424951919632)\n" +
 					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
 					"CarEvent(1,50,26.38888888888889,1424951919632)\n" +
 					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
-					"CarEvent(1,55,159.7222222222222,1424951930637)\n" +
+					"CarEvent(1,65,194.4444444444444,1424951932638)\n" +
 					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
 					"CarEvent(1,70,213.88888888888886,1424951933638)\n" +
-					"CarEvent(0,60,156.94444444444443,1424951927636)\n" +
+					"CarEvent(0,60,218.05555555555551,1424951931637)\n" +
 					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
+					"CarEvent(0,55,233.3333333333333,1424951932637)\n" +
 					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
-					"CarEvent(0,60,218.05555555555551,1424951931637)\n" +
 					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
 					"CarEvent(0,55,288.88888888888886,1424951936639)\n" +
-					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
-					"CarEvent(1,75,497.2222222222222,1424951948643)\n" +
-					"CarEvent(0,55,344.44444444444446,1424951940640)\n" +
+					"CarEvent(1,70,329.16666666666663,1424951939640)\n" +
+					"CarEvent(0,55,373.61111111111114,1424951942641)\n" +
 					"CarEvent(1,80,519.4444444444443,1424951949644)\n" +
-					"CarEvent(0,50,413.88888888888897,1424951945642)\n" +
 					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
+					"CarEvent(0,50,487.50000000000006,1424951951644)\n" +
 					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
-					"CarEvent(0,55,573.6111111111112,1424951958646)\n" +
+					"CarEvent(0,60,590.2777777777778,1424951959647)\n" +
 					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
-					"CarEvent(0,70,627.7777777777778,1424951961647)\n" +
-					"CarEvent(1,90,831.9444444444441,1424951963648)\n" +
+					"CarEvent(0,75,648.6111111111112,1424951962648)\n" +
+					"CarEvent(1,85,715.2777777777776,1424951958647)\n" +
+					"CarEvent(1,95,858.333333333333,1424951964649)\n" +
 					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
 					"CarEvent(1,95,858.333333333333,1424951964649)\n" +
 					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
@@ -233,22 +233,22 @@ public class TopSpeedWindowingExampleData {
 					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
 					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
 					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
-					"CarEvent(1,100,965.2777777777776,1424951968650)\n" +
+					"CarEvent(1,100,993.0555555555554,1424951969650)\n" +
 					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
-					"CarEvent(1,100,1020.8333333333333,1424951970651)\n" +
-					"CarEvent(1,100,1076.388888888889,1424951972651)\n" +
-					"CarEvent(0,90,1058.3333333333335,1424951981654)\n" +
+					"CarEvent(1,100,1048.611111111111,1424951971651)\n" +
 					"CarEvent(1,100,1130.5555555555557,1424951974652)\n" +
-					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
-					"CarEvent(1,100,1186.1111111111113,1424951976653)\n" +
+					"CarEvent(0,90,1058.3333333333335,1424951981654)\n" +
+					"CarEvent(1,100,1158.3333333333335,1424951975652)\n" +
 					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
 					"CarEvent(1,100,1240.277777777778,1424951978653)\n" +
 					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
-					"CarEvent(1,100,1295.8333333333337,1424951980654)\n" +
+					"CarEvent(1,100,1268.0555555555559,1424951979654)\n" +
+					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
+					"CarEvent(1,100,1323.6111111111115,1424951981654)\n" +
 					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
-					"CarEvent(1,100,1351.3888888888894,1424951982655)\n" +
+					"CarEvent(1,100,1379.1666666666672,1424951983655)\n" +
 					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
-					"CarEvent(1,100,1406.944444444445,1424951984656)\n" +
+					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
 					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
 					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
 					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
@@ -256,17 +256,17 @@ public class TopSpeedWindowingExampleData {
 					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
 					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
 					"CarEvent(1,100,1669.4444444444453,1424951994659)\n" +
-					"CarEvent(0,100,1386.1111111111113,1424951994659)\n" +
-					"CarEvent(1,95,1695.8333333333342,1424951995660)\n" +
 					"CarEvent(0,100,1440.277777777778,1424951996660)\n" +
-					"CarEvent(1,90,1947.2222222222226,1424952006664)\n" +
+					"CarEvent(1,90,1720.8333333333342,1424951996660)\n" +
+					"CarEvent(0,100,1468.0555555555559,1424951997660)\n" +
+					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
 					"CarEvent(0,100,1522.2222222222226,1424951999661)\n" +
 					"CarEvent(0,100,1627.7777777777783,1424952003662)\n" +
 					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
 					"CarEvent(0,100,1627.7777777777783,1424952003662)\n" +
 					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
-					"CarEvent(0,100,1655.555555555556,1424952004663)\n" +
 					"CarEvent(0,100,1709.7222222222229,1424952006663)\n" +
+					"CarEvent(0,100,1737.5000000000007,1424952007664)\n" +
 					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
 					"CarEvent(0,100,1791.6666666666674,1424952009664)\n" +
 					"CarEvent(1,95,2211.1111111111118,1424952017668)\n";

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
index 8e3c7d6..1419afd 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
@@ -19,8 +19,15 @@
 package org.apache.flink.streaming.scala.examples.windowing
 
 
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction
 import org.apache.flink.streaming.api.scala._
-import org.apache.flink.streaming.api.scala.windowing.{Delta, Time}
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
+import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger
 
 import scala.Stream._
 import scala.math._
@@ -53,16 +60,23 @@ object TopSpeedWindowing {
     }
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setParallelism(1)
 
     val cars = setCarsInput(env)
 
-    val topSeed = cars.keyBy("carId")
-      .window(Time.of(evictionSec * 1000, (car : CarEvent) => car.time))
-      .every(Delta.of[CarEvent](triggerMeters,
-          (oldSp,newSp) => newSp.distance-oldSp.distance, CarEvent(0,0,0,0)))
-      .local    
+    val topSeed = cars
+      .extractAscendingTimestamp( _.time )
+      .keyBy("carId")
+      .window(GlobalWindows.create)
+      .evictor(TimeEvictor.of(Time.of(evictionSec * 1000, TimeUnit.MILLISECONDS)))
+      .trigger(DeltaTrigger.of(triggerMeters, new DeltaFunction[CarEvent] {
+        def getDelta(oldSp: CarEvent, newSp: CarEvent): Double = newSp.distance - oldSp.distance
+      }))
+//      .window(Time.of(evictionSec * 1000, (car : CarEvent) => car.time))
+//      .every(Delta.of[CarEvent](triggerMeters,
+//          (oldSp,newSp) => newSp.distance-oldSp.distance, CarEvent(0,0,0,0)))
       .maxBy("speed")
-      .flatten()
 
     if (fileOutput) {
       topSeed.writeAsText(outputPath)

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
index 65cafb7..33104ab 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
@@ -27,9 +27,12 @@ import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
 import org.apache.flink.streaming.api.windowing.evictors.Evictor
 import org.apache.flink.streaming.api.windowing.triggers.Trigger
 import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.Collector
 
 import scala.reflect.ClassTag
 
+import scala.collection.JavaConverters._
+
 /**
  * A [[AllWindowedStream]] represents a data stream where the stream of
  * elements is split into windows based on a
@@ -177,6 +180,28 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
     javaStream.apply(clean(function), implicitly[TypeInformation[R]])
   }
 
+  /**
+   * Applies the given window function to each window. The window function is called for each
+   * evaluation of the window for each key individually. The output of the window function is
+   * interpreted as a regular non-windowed stream.
+   *
+   * Not that this function requires that all data in the windows is buffered until the window
+   * is evaluated, as the function provides no means of pre-aggregation.
+   *
+   * @param function The window function.
+   * @return The data stream that is the result of applying the window function to the window.
+   */
+  def apply[R: TypeInformation: ClassTag](
+      function: (W, Iterable[T], Collector[R]) => Unit): DataStream[R] = {
+    val cleanedFunction = clean(function)
+    val applyFunction = new AllWindowFunction[T, R, W] {
+      def apply(window: W, elements: java.lang.Iterable[T], out: Collector[R]): Unit = {
+        cleanedFunction(window, elements.asScala, out)
+      }
+    }
+    javaStream.apply(applyFunction, implicitly[TypeInformation[R]])
+  }
+
   // ------------------------------------------------------------------------
   //  Aggregations on the keyed windows
   // ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index fb4d75d..22abbdf 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -32,8 +32,6 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction
 import org.apache.flink.streaming.api.functions.{AscendingTimestampExtractor, TimestampExtractor}
 import org.apache.flink.streaming.api.scala.function.StatefulFunction
 import org.apache.flink.streaming.api.windowing.assigners._
-import org.apache.flink.streaming.api.windowing.helper.WindowingHelper
-import org.apache.flink.streaming.api.windowing.policy.{EvictionPolicy, TriggerPolicy}
 import org.apache.flink.streaming.api.windowing.time.AbstractTime
 import org.apache.flink.streaming.api.windowing.windows.{TimeWindow, Window}
 import org.apache.flink.streaming.util.serialization.SerializationSchema
@@ -105,8 +103,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
    */
   def name(name: String) : DataStream[T] = javaStream match {
     case stream : SingleOutputStreamOperator[T,_] => stream.name(name)
-    case _ => throw new
-        UnsupportedOperationException("Only supported for operators.")
+    case _ => throw new UnsupportedOperationException("Only supported for operators.")
     this
   }
   
@@ -583,37 +580,6 @@ class DataStream[T](javaStream: JavaStream[T]) {
   }
 
   /**
-   * Create a WindowedDataStream that can be used to apply
-   * transformation like .reduceWindow(...) or aggregations on
-   * preset chunks(windows) of the data stream. To define the windows a
-   * WindowingHelper such as Time, Count and
-   * Delta can be used.</br></br> When applied to a grouped data
-   * stream, the windows (evictions) and slide sizes (triggers) will be
-   * computed on a per group basis. </br></br> For more advanced control over
-   * the trigger and eviction policies please use to
-   * window(List(triggers), List(evicters))
-   */
-  def window(windowingHelper: WindowingHelper[_]): WindowedDataStream[T] =
-    javaStream.window(windowingHelper)
-
-  /**
-   * Create a WindowedDataStream using the given Trigger and Eviction policies.
-   * Windowing can be used to apply transformation like .reduceWindow(...) or 
-   * aggregations on preset chunks(windows) of the data stream.</br></br>For most common
-   * use-cases please refer to window(WindowingHelper[_])
-   *
-   */
-  def window(trigger: TriggerPolicy[T], eviction: EvictionPolicy[T]):
-    WindowedDataStream[T] = javaStream.window(trigger, eviction)
-    
-  /**
-   * Create a WindowedDataStream based on the full stream history to perform periodic
-   * aggregations.
-   */  
-  def every(windowingHelper: WindowingHelper[_]): WindowedDataStream[T] = 
-    javaStream.every(windowingHelper)
-
-  /**
    * Windows this DataStream into tumbling time windows.
    *
    * This is a shortcut for either `.window(TumblingTimeWindows.of(size))` or

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala
deleted file mode 100644
index 8ef94f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala
+++ /dev/null
@@ -1,338 +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.flink.streaming.api.scala
-
-import org.apache.flink.api.scala.ClosureCleaner
-
-import scala.Array.canBuildFrom
-import scala.collection.JavaConverters._
-import scala.reflect.ClassTag
-
-import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.streaming.api.datastream.{WindowedDataStream => JavaWStream, DiscretizedStream}
-import org.apache.flink.streaming.api.functions.WindowMapFunction
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
-import org.apache.flink.streaming.api.windowing.StreamWindow
-import org.apache.flink.streaming.api.windowing.helper.WindowingHelper
-import org.apache.flink.util.Collector
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator
-
-class WindowedDataStream[T](javaStream: JavaWStream[T]) {
-
-  /**
-   * Gets the name of the current data stream. This name is
-   * used by the visualization and logging during runtime.
-   *
-   * @return Name of the stream.
-   */
-  def getName : String = javaStream match {
-    case stream : DiscretizedStream[_] => stream.getName
-    case _ => throw new
-        UnsupportedOperationException("Only supported for windowing operators.")
-  }
-
-  /**
-   * Sets the name of the current data stream. This name is
-   * used by the visualization and logging during runtime.
-   *
-   * @return The named operator
-   */
-  def name(name: String) : WindowedDataStream[T] = javaStream match {
-    case stream : DiscretizedStream[T] => stream.name(name)
-    case _ => throw new
-        UnsupportedOperationException("Only supported for windowing operators.")
-    this
-  }
-
-  /**
-   * Defines the slide size (trigger frequency) for the windowed data stream.
-   * This controls how often the user defined function will be triggered on
-   * the window.
-   */
-  def every(windowingHelper: WindowingHelper[_]): WindowedDataStream[T] =
-    javaStream.every(windowingHelper)
-
-  /**
-   * Groups the elements of the WindowedDataStream using the given
-   * field positions. The window sizes (evictions) and slide sizes
-   * (triggers) will be calculated on the whole stream (in a global fashion),
-   * but the user defined functions will be applied on a per group basis.
-   * </br></br> To get windows and triggers on a per group basis apply the
-   * DataStream.window(...) operator on an already grouped data stream.
-   *
-   */
-  def keyBy(fields: Int*): WindowedDataStream[T] = javaStream.keyBy(fields: _*)
-
-  /**
-   * Groups the elements of the WindowedDataStream using the given
-   * field expressions. The window sizes (evictions) and slide sizes
-   * (triggers) will be calculated on the whole stream (in a global fashion),
-   * but the user defined functions will be applied on a per group basis.
-   * </br></br> To get windows and triggers on a per group basis apply the
-   * DataStream.window(...) operator on an already grouped data stream.
-   *
-   */
-  def keyBy(firstField: String, otherFields: String*): WindowedDataStream[T] =
-   javaStream.keyBy(firstField +: otherFields.toArray: _*)
-    
-  /**
-   * Groups the elements of the WindowedDataStream using the given
-   * KeySelector function. The window sizes (evictions) and slide sizes
-   * (triggers) will be calculated on the whole stream (in a global fashion),
-   * but the user defined functions will be applied on a per group basis.
-   * </br></br> To get windows and triggers on a per group basis apply the
-   * DataStream.window(...) operator on an already grouped data stream.
-   *
-   */
-  def keyBy[K: TypeInformation](fun: T => K): WindowedDataStream[T] = {
-
-    val cleanFun = clean(fun)
-    val keyExtractor = new KeySelector[T, K] {
-      def getKey(in: T) = cleanFun(in)
-    }
-    javaStream.keyBy(keyExtractor)
-  }
-  
-  /**
-   * Sets the window discretisation local, meaning that windows will be
-   * created in parallel at environment parallelism.
-   * 
-   */
-  def local(): WindowedDataStream[T] = javaStream.local
- 
-  /**
-   * Flattens the result of a window transformation returning the stream of window
-   * contents elementwise.
-   */
-  def flatten(): DataStream[T] = javaStream.flatten()
-  
-  /**
-   * Returns the stream of StreamWindows created by the window tranformation
-   */
-  def getDiscretizedStream(): DataStream[StreamWindow[T]] = javaStream.getDiscretizedStream()
-
-  /**
-   * Applies a reduce transformation on the windowed data stream by reducing
-   * the current window at every trigger.
-   *
-   */
-  def reduceWindow(reducer: ReduceFunction[T]): WindowedDataStream[T] = {
-    if (reducer == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    javaStream.reduceWindow(reducer)
-  }
-
-  /**
-   * Applies a reduce transformation on the windowed data stream by reducing
-   * the current window at every trigger.
-   *
-   */
-  def reduceWindow(fun: (T, T) => T): WindowedDataStream[T] = {
-    if (fun == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val reducer = new ReduceFunction[T] {
-      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
-    }
-    reduceWindow(reducer)
-  }
-
-  /**
-   * Applies a fold transformation on the windowed data stream by reducing
-   * the current window at every trigger.
-   *
-   */
-  def foldWindow[R: TypeInformation: ClassTag](initialValue: R, folder: FoldFunction[T,R]): 
-  WindowedDataStream[R] = {
-    if (folder == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    javaStream.foldWindow(initialValue, folder, implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies a fold transformation on the windowed data stream by reducing
-   * the current window at every trigger.
-   *
-   */
-  def foldWindow[R: TypeInformation: ClassTag](initialValue: R, fun: (R, T) => R):
-  WindowedDataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val folder = new FoldFunction[T,R] {
-      def fold(acc: R, v: T) = { cleanFun(acc, v) }
-    }
-    foldWindow(initialValue, folder)
-  }
-
-  /**
-   * Applies a mapWindow transformation on the windowed data stream by calling the mapWindow
-   * method on current window at every trigger. In contrast with the simple binary reduce 
-   * operator, mapWindow exposes the whole window through the Iterable interface.
-   * </br>
-   * </br>
-   * Whenever possible try to use reduceWindow instead of mapWindow for increased efficiency
-   */
-  def mapWindow[R: ClassTag: TypeInformation](reducer: WindowMapFunction[T, R]):
-  WindowedDataStream[R] = {
-    if (reducer == null) {
-      throw new NullPointerException("GroupReduce function must not be null.")
-    }
-    javaStream.mapWindow(reducer, implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies a mapWindow transformation on the windowed data stream by calling the mapWindow
-   * method on current window at every trigger. In contrast with the simple binary reduce 
-   * operator, mapWindow exposes the whole window through the Iterable interface.
-   * </br>
-   * </br>
-   * Whenever possible try to use reduceWindow instead of mapWindow for increased efficiency
-   */
-  def mapWindow[R: ClassTag: TypeInformation](fun: (Iterable[T], Collector[R]) => Unit):
-  WindowedDataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("GroupReduce function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val reducer = new WindowMapFunction[T, R] {
-      def mapWindow(in: java.lang.Iterable[T], out: Collector[R]) = { cleanFun(in.asScala, out) }
-    }
-    mapWindow(reducer)
-  }
-
-  /**
-   * Applies an aggregation that that gives the maximum of the elements in the window at
-   * the given position.
-   *
-   */
-  def max(position: Int): WindowedDataStream[T] = aggregate(AggregationType.MAX, position)
-  
-  /**
-   * Applies an aggregation that that gives the maximum of the elements in the window at
-   * the given field.
-   *
-   */
-  def max(field: String): WindowedDataStream[T] = aggregate(AggregationType.MAX, field)
-
-  /**
-   * Applies an aggregation that that gives the minimum of the elements in the window at
-   * the given position.
-   *
-   */
-  def min(position: Int): WindowedDataStream[T] = aggregate(AggregationType.MIN, position)
-  
-  /**
-   * Applies an aggregation that that gives the minimum of the elements in the window at
-   * the given field.
-   *
-   */
-  def min(field: String): WindowedDataStream[T] = aggregate(AggregationType.MIN, field)
-
-  /**
-   * Applies an aggregation that sums the elements in the window at the given position.
-   *
-   */
-  def sum(position: Int): WindowedDataStream[T] = aggregate(AggregationType.SUM, position)
-  
-  /**
-   * Applies an aggregation that sums the elements in the window at the given field.
-   *
-   */
-  def sum(field: String): WindowedDataStream[T] = aggregate(AggregationType.SUM, field)
-
-  /**
-   * Applies an aggregation that that gives the maximum element of the window by
-   * the given position. When equality, returns the first.
-   *
-   */
-  def maxBy(position: Int): WindowedDataStream[T] = aggregate(AggregationType.MAXBY,
-    position)
-    
-  /**
-   * Applies an aggregation that that gives the maximum element of the window by
-   * the given field. When equality, returns the first.
-   *
-   */
-  def maxBy(field: String): WindowedDataStream[T] = aggregate(AggregationType.MAXBY,
-    field)
-
-  /**
-   * Applies an aggregation that that gives the minimum element of the window by
-   * the given position. When equality, returns the first.
-   *
-   */
-  def minBy(position: Int): WindowedDataStream[T] = aggregate(AggregationType.MINBY,
-    position)
-    
-   /**
-   * Applies an aggregation that that gives the minimum element of the window by
-   * the given field. When equality, returns the first.
-   *
-   */
-  def minBy(field: String): WindowedDataStream[T] = aggregate(AggregationType.MINBY,
-    field)
-    
-  private def aggregate(aggregationType: AggregationType, field: String): 
-  WindowedDataStream[T] = {
-    val position = fieldNames2Indices(getType(), Array(field))(0)
-    aggregate(aggregationType, position)
-  }  
-
-  def aggregate(aggregationType: AggregationType, position: Int):
-  WindowedDataStream[T] = {
-
-    val jStream = javaStream.asInstanceOf[JavaWStream[Product]]
-
-    val reducer = aggregationType match {
-      case AggregationType.SUM =>
-        new SumAggregator(position, jStream.getType, jStream.getExecutionConfig)
-      case _ =>
-        new ComparableAggregator(position, jStream.getType, aggregationType, true,
-          jStream.getExecutionConfig)
-    }
-
-    new WindowedDataStream[Product](
-            jStream.reduceWindow(reducer)).asInstanceOf[WindowedDataStream[T]]
-  }
-
-  /**
-   * Gets the output type.
-   *
-   * @return The output type.
-   */
-  def getType(): TypeInformation[T] = javaStream.getType
-
-  /**
-   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig}
-   */
-  private[flink] def clean[F <: AnyRef](f: F): F = {
-    new StreamExecutionEnvironment(
-      javaStream.getDiscretizedStream.getExecutionEnvironment).scalaClean(f)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
index a8ddaf8..d4f4618 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
@@ -27,9 +27,12 @@ import org.apache.flink.streaming.api.functions.windowing.WindowFunction
 import org.apache.flink.streaming.api.windowing.evictors.Evictor
 import org.apache.flink.streaming.api.windowing.triggers.Trigger
 import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.Collector
 
 import scala.reflect.ClassTag
 
+import scala.collection.JavaConverters._
+
 /**
  * A [[WindowedStream]] represents a data stream where elements are grouped by
  * key, and for each key, the stream of elements is split into windows based on a
@@ -180,6 +183,28 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
     javaStream.apply(clean(function), implicitly[TypeInformation[R]])
   }
 
+  /**
+   * Applies the given window function to each window. The window function is called for each
+   * evaluation of the window for each key individually. The output of the window function is
+   * interpreted as a regular non-windowed stream.
+   *
+   * Not that this function requires that all data in the windows is buffered until the window
+   * is evaluated, as the function provides no means of pre-aggregation.
+   *
+   * @param function The window function.
+   * @return The data stream that is the result of applying the window function to the window.
+   */
+  def apply[R: TypeInformation: ClassTag](
+      function: (K, W, Iterable[T], Collector[R]) => Unit): DataStream[R] = {
+    val cleanedFunction = clean(function)
+    val applyFunction = new WindowFunction[T, R, K, W] {
+      def apply(key: K, window: W, elements: java.lang.Iterable[T], out: Collector[R]): Unit = {
+        cleanedFunction(key, window, elements.asScala, out)
+      }
+    }
+    javaStream.apply(applyFunction, implicitly[TypeInformation[R]])
+  }
+
   // ------------------------------------------------------------------------
   //  Aggregations on the keyed windows
   // ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
index d65ea41..e668064 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
@@ -24,7 +24,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala.{createTuple2TypeInformation => apiTupleCreator}
 import org.apache.flink.api.scala.typeutils.{CaseClassTypeInfo, TypeUtils}
 import org.apache.flink.streaming.api.datastream.{ DataStream => JavaStream }
-import org.apache.flink.streaming.api.datastream.{ WindowedDataStream => JavaWStream }
 import org.apache.flink.streaming.api.datastream.{ SplitStream => SplitJavaStream }
 import org.apache.flink.streaming.api.datastream.{ ConnectedStreams => ConnectedJavaStreams }
 import org.apache.flink.streaming.api.datastream.{ KeyedStream => KeyedJavaStream }
@@ -41,9 +40,6 @@ package object scala {
   implicit def javaToScalaGroupedStream[R, K](javaStream: KeyedJavaStream[R, K]):
   KeyedStream[R, K] = new KeyedStream[R, K](javaStream)
 
-  implicit def javaToScalaWindowedStream[R](javaWStream: JavaWStream[R]): WindowedDataStream[R] =
-    new WindowedDataStream[R](javaWStream)
-
   implicit def javaToScalaSplitStream[R](javaStream: SplitJavaStream[R]): SplitStream[R] =
     new SplitStream[R](javaStream)
 
@@ -75,7 +71,6 @@ package object scala {
 
   def createTuple2TypeInformation[T1, T2](
       t1: TypeInformation[T1],
-      t2: TypeInformation[T2])
-    : TypeInformation[(T1, T2)] =
+      t2: TypeInformation[T2]) : TypeInformation[(T1, T2)] =
     apiTupleCreator[T1, T2](t1, t2)
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Delta.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Delta.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Delta.scala
deleted file mode 100644
index 461ad3c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Delta.scala
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala.windowing
-
-import org.apache.flink.api.scala.ClosureCleaner
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction
-import org.apache.flink.streaming.api.windowing.helper.{ Delta => JavaDelta }
-
-object Delta {
-
-  /**
-   * Creates a delta helper representing a delta trigger or eviction policy.
-   * </br></br> This policy calculates a delta between the data point which
-   * triggered last and the currently arrived data point. It triggers if the
-   * delta is higher than a specified threshold. </br></br> In case it gets
-   * used for eviction, this policy starts from the first element of the
-   * buffer and removes all elements from the buffer which have a higher delta
-   * then the threshold. As soon as there is an element with a lower delta,
-   * the eviction stops.
-   */
-  def of[T](threshold: Double, deltaFunction: (T, T) => Double, initVal: T): JavaDelta[T] = {
-    require(deltaFunction != null, "Delta function must not be null")
-    val df = new DeltaFunction[T] {
-      ClosureCleaner.clean(deltaFunction, true)
-      override def getDelta(first: T, second: T) = deltaFunction(first, second)
-    }
-    JavaDelta.of(threshold, df, initVal)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Time.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Time.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Time.scala
deleted file mode 100644
index 5cea95b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/windowing/Time.scala
+++ /dev/null
@@ -1,53 +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.flink.streaming.api.scala.windowing
-
-import java.util.concurrent.TimeUnit
-import org.apache.flink.api.scala.ClosureCleaner
-import org.apache.flink.streaming.api.windowing.helper.{ Time => JavaTime }
-
-import org.apache.flink.streaming.api.windowing.helper.Timestamp
-
-object Time {
-
-  /**
-   * Creates a helper representing a time trigger which triggers every given
-   * length (slide size) or a time eviction which evicts all elements older
-   * than length (window size) using System time.
-   *
-   */
-  def of(windowSize: Long, timeUnit: TimeUnit): JavaTime[_] =
-    JavaTime.of(windowSize, timeUnit)
-
-  /**
-   * Creates a helper representing a time trigger which triggers every given
-   * length (slide size) or a time eviction which evicts all elements older
-   * than length (window size) using a user defined timestamp extractor.
-   *
-   */
-  def of[R](windowSize: Long, timestamp: R => Long, startTime: Long = 0): JavaTime[R] = {
-    require(timestamp != null, "Timestamp must not be null.")
-    val ts = new Timestamp[R] {
-      ClosureCleaner.clean(timestamp, true)
-      override def getTimestamp(in: R) = timestamp(in)
-    }
-    JavaTime.of(windowSize, ts, startTime)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
index 5a5a8c9..91639ed 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
@@ -19,14 +19,15 @@
 package org.apache.flink.streaming.api.scala
 
 import java.lang
-import org.apache.flink.api.common.functions.{FilterFunction, FlatMapFunction, MapFunction,
-  Partitioner, FoldFunction, Function}
+import org.apache.flink.api.common.functions.{FilterFunction, FlatMapFunction, MapFunction, Partitioner, FoldFunction, Function}
 import org.apache.flink.api.java.typeutils.TypeExtractor
 import org.apache.flink.streaming.api.collector.selector.OutputSelector
 import org.apache.flink.streaming.api.functions.co.CoMapFunction
 import org.apache.flink.streaming.api.graph.{StreamEdge, StreamGraph}
 import org.apache.flink.streaming.api.operators.{AbstractUdfStreamOperator, StreamOperator}
-import org.apache.flink.streaming.api.windowing.helper.Count
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
+import org.apache.flink.streaming.api.windowing.triggers.{PurgingTrigger, CountTrigger}
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow
 import org.apache.flink.streaming.runtime.partitioner._
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.util.Collector
@@ -56,21 +57,24 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     assert("testReduce" == dataStream2.getName)
 
     val connected = dataStream1.connect(dataStream2)
-      .flatMap(
-    { (in, out: Collector[Long]) => }, { (in, out: Collector[Long]) => }
-    ).name("testCoFlatMap")
+      .flatMap({ (in, out: Collector[(Long, Long)]) => }, { (in, out: Collector[(Long, Long)]) => })
+      .name("testCoFlatMap")
+
     assert("testCoFlatMap" == connected.getName)
 
-    val func: ((Long, Long) => Long) =
-      (x: Long, y: Long) => 0L
+    val func: (((Long, Long), (Long, Long)) => (Long, Long)) =
+      (x: (Long, Long), y: (Long, Long)) => (0L, 0L)
 
-    val windowed = connected.window(Count.of(10))
-      .foldWindow(0L, func)
+    val windowed = connected
+      .windowAll(GlobalWindows.create())
+      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](10)))
+      .fold((0L, 0L), func)
 
     windowed.name("testWindowFold")
+
     assert("testWindowFold" == windowed.getName)
 
-    windowed.flatten().print()
+    windowed.print()
 
     val plan = env.getExecutionPlan
 
@@ -239,11 +243,12 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     val env: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment(10)
 
     val src = env.fromElements(new Tuple2[Long, Long](0L, 0L))
-    val map = src.map(x => 0L)
-    val windowed: DataStream[Long] = map
-      .window(Count.of(10))
-      .foldWindow(0L, (x: Long, y: Long) => 0L)
-      .flatten
+    val map = src.map(x => (0L, 0L))
+    val windowed: DataStream[(Long, Long)] = map
+      .windowAll(GlobalWindows.create())
+      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](10)))
+      .fold((0L, 0L), (x: (Long, Long), y: (Long, Long)) => (0L, 0L))
+
     windowed.print()
     val sink = map.addSink(x => {})
 
@@ -294,15 +299,17 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
     val map: DataStream[(Integer, String)] = src1.map(x => null)
     assert(classOf[scala.Tuple2[Integer, String]] == map.getType.getTypeClass)
 
-    val window: WindowedDataStream[String] = map
-      .window(Count.of(5))
-      .mapWindow((x: Iterable[(Integer, String)], y: Collector[String]) => {})
+    val window: DataStream[String] = map
+      .windowAll(GlobalWindows.create())
+      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](5)))
+      .apply((w: GlobalWindow, x: Iterable[(Integer, String)], y: Collector[String]) => {})
+
     assert(TypeExtractor.getForClass(classOf[String]) == window.getType)
 
     val flatten: DataStream[Int] = window
-      .foldWindow(0,
-        (accumulator: Int, value: String) => 0
-      ).flatten
+      .windowAll(GlobalWindows.create())
+      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](5)))
+      .fold(0, (accumulator: Int, value: String) => 0)
     assert(TypeExtractor.getForClass(classOf[Int]) == flatten.getType)
 
     // TODO check for custom case class

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
index 53aa1e2..101f3b5 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.scala
 import java.lang.reflect.Method
 
 import org.apache.flink.api.scala.completeness.ScalaAPICompletenessTestBase
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, JoinedStreams}
+import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream}
 
 import scala.language.existentials
 
@@ -149,10 +149,5 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
       "CoGroupedStreams.WithWindow", "CoGroupedStreams.WithWindow",
       classOf[org.apache.flink.streaming.api.datastream.CoGroupedStreams.WithWindow[_,_,_,_]],
       classOf[CoGroupedStreams.WithWindow[_,_,_,_]])
-
-    checkMethods(
-      "WindowedDataStream", "WindowedDataStream",
-      classOf[org.apache.flink.streaming.api.datastream.WindowedDataStream[_]],
-      classOf[WindowedDataStream[_]])
   }
 }


[05/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducer.java
deleted file mode 100644
index 8d690cc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducer.java
+++ /dev/null
@@ -1,83 +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.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-
-public class SlidingCountGroupedPreReducer<T> extends SlidingGroupedPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private long windowSize;
-	private long slideSize;
-	private int start;
-
-	protected long index = 0;
-
-	public SlidingCountGroupedPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer,
-			KeySelector<T, ?> key, long windowSize, long slideSize, int start) {
-		super(reducer, serializer, key);
-		if (windowSize > slideSize) {
-			this.windowSize = windowSize;
-			this.slideSize = slideSize;
-			this.start = start;
-		} else {
-			throw new RuntimeException(
-					"Window size needs to be larger than slide size for the sliding pre-reducer");
-		}
-		index = index - start;
-	}
-
-	@Override
-	protected void afterStore() {
-		index++;
-	}
-
-	@Override
-	public void store(T element) throws Exception {
-		if (index >= 0) {
-			super.store(element);
-		} else {
-			index++;
-		}
-	}
-
-	@Override
-	protected boolean currentEligible(T next) {
-		if (index <= slideSize) {
-			return true;
-		} else {
-			return index == windowSize;
-		}
-	}
-
-	@Override
-	protected void afterEmit() {
-		if (index >= slideSize) {
-			index = index - slideSize;
-		}
-	}
-
-	@Override
-	public SlidingCountGroupedPreReducer<T> clone() {
-		return new SlidingCountGroupedPreReducer<T>(reducer, serializer, key, windowSize,
-				slideSize, start);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducer.java
deleted file mode 100644
index db14eb0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducer.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-public class SlidingCountPreReducer<T> extends SlidingPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private long windowSize;
-	private long slideSize;
-	private int start;
-
-	protected long index = 0;
-
-	public SlidingCountPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer,
-			long windowSize, long slideSize, int start) {
-		super(reducer, serializer);
-		if (windowSize > slideSize) {
-			this.windowSize = windowSize;
-			this.slideSize = slideSize;
-			this.start = start;
-		} else {
-			throw new RuntimeException(
-					"Window size needs to be larger than slide size for the sliding pre-reducer");
-		}
-		index = index - start;
-	}
-
-	@Override
-	protected void afterStore() {
-		index++;
-	}
-
-	@Override
-	public void store(T element) throws Exception {
-		if (index >= 0) {
-			super.store(element);
-		} else {
-			index++;
-		}
-	}
-
-	@Override
-	protected boolean currentEligible(T next) {
-		if (index <= slideSize) {
-			return true;
-		} else {
-			return index == windowSize;
-		}
-	}
-
-	@Override
-	protected void afterEmit() {
-		if (index >= slideSize) {
-			index = index - slideSize;
-		}
-	}
-
-	@Override
-	public SlidingCountPreReducer<T> clone() {
-		return new SlidingCountPreReducer<T>(reducer, serializer, windowSize, slideSize, start);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingGroupedPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingGroupedPreReducer.java
deleted file mode 100644
index 6e5462c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingGroupedPreReducer.java
+++ /dev/null
@@ -1,153 +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.flink.streaming.api.windowing.windowbuffer;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-
-/**
- * Grouped pre-reducer for sliding eviction policy
- * (the slide size is smaller than the window size).
- */
-public abstract class SlidingGroupedPreReducer<T> extends SlidingPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected Map<Object, T> currentReducedMap = new HashMap<Object, T>();
-	protected LinkedList<Map<Object, T>> reducedMap = new LinkedList<Map<Object, T>>();
-
-	protected KeySelector<T, ?> key;
-
-	public SlidingGroupedPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer,
-			KeySelector<T, ?> key) {
-		super(reducer, serializer);
-		this.key = key;
-	}
-
-	public boolean addFinalAggregate(StreamWindow<T> currentWindow) throws Exception {
-		Map<Object, T> finalReduce = null;
-
-		if (!reducedMap.isEmpty()) {
-			finalReduce = reducedMap.get(0);
-			for (int i = 1; i < reducedMap.size(); i++) {
-				finalReduce = reduceMaps(finalReduce, reducedMap.get(i));
-
-			}
-			if (currentReducedMap != null) {
-				finalReduce = reduceMaps(finalReduce, currentReducedMap);
-			}
-
-		} else {
-			finalReduce = currentReducedMap;
-		}
-
-		if (finalReduce != null) {
-			currentWindow.addAll(finalReduce.values());
-			return true;
-		} else {
-			return false;
-		}
-
-	}
-
-	private Map<Object, T> reduceMaps(Map<Object, T> first, Map<Object, T> second) throws Exception {
-
-		Map<Object, T> reduced = new HashMap<Object, T>();
-
-		// Get the common keys in the maps
-		Set<Object> interSection = new HashSet<Object>();
-		Set<Object> diffFirst = new HashSet<Object>();
-		Set<Object> diffSecond = new HashSet<Object>();
-
-		for (Object key : first.keySet()) {
-			if (second.containsKey(key)) {
-				interSection.add(key);
-			} else {
-				diffFirst.add(key);
-			}
-		}
-
-		for (Object key : second.keySet()) {
-			if (!interSection.contains(key)) {
-				diffSecond.add(key);
-			}
-		}
-
-		// Reduce the common keys
-		for (Object key : interSection) {
-			reduced.put(
-					key,
-					reducer.reduce(serializer.copy(first.get(key)),
-							serializer.copy(second.get(key))));
-		}
-
-		for (Object key : diffFirst) {
-			reduced.put(key, first.get(key));
-		}
-
-		for (Object key : diffSecond) {
-			reduced.put(key, second.get(key));
-		}
-
-		return reduced;
-	}
-
-	protected void updateCurrent(T element) throws Exception {
-		if (currentReducedMap == null) {
-			currentReducedMap = new HashMap<Object, T>();
-			currentReducedMap.put(key.getKey(element), element);
-		} else {
-			Object nextKey = key.getKey(element);
-			T last = currentReducedMap.get(nextKey);
-			if (last == null) {
-				currentReducedMap.put(nextKey, element);
-			} else {
-				currentReducedMap.put(nextKey, reducer.reduce(serializer.copy(last), element));
-			}
-		}
-	}
-
-	@Override
-	protected void removeLastReduced() {
-		reducedMap.removeFirst();
-	}
-
-	@Override
-	protected void addCurrentToBuffer(T element) throws Exception {
-		reducedMap.add(currentReducedMap);
-	}
-
-	@Override
-	protected void resetCurrent() {
-		currentReducedMap = null;
-		elementsSinceLastPreAggregate = 0;
-	}
-
-	@Override
-	protected boolean currentNotEmpty() {
-		return currentReducedMap != null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java
deleted file mode 100644
index e2c46a3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingPreReducer.java
+++ /dev/null
@@ -1,175 +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.flink.streaming.api.windowing.windowbuffer;
-
-import java.util.LinkedList;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-/**
- * Non-grouped pre-reducer for sliding eviction policy
- * (the slide size is smaller than the window size).
- */
-public abstract class SlidingPreReducer<T> extends WindowBuffer<T> implements PreAggregator {
-
-	private static final long serialVersionUID = 1L;
-
-	protected ReduceFunction<T> reducer;
-
-	protected T currentReduced;
-	protected LinkedList<T> reduced = new LinkedList<T>();
-	protected LinkedList<Integer> elementsPerPreAggregate = new LinkedList<Integer>();
-
-	protected TypeSerializer<T> serializer;
-
-	protected int toRemove = 0;
-
-	protected int elementsSinceLastPreAggregate = 0;
-
-	public SlidingPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer) {
-		this.reducer = reducer;
-		this.serializer = serializer;
-	}
-
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-		StreamWindow<T> currentWindow = createEmptyWindow();
-
-		try {
-			if (addFinalAggregate(currentWindow) || emitEmpty) {
-				collector.collect(new StreamRecord<StreamWindow<T>>(currentWindow));
-			} 
-			afterEmit();
-		} catch (Exception e) {
-			throw new RuntimeException(e);
-		}
-
-	}
-
-	protected void afterEmit() {
-		// Do nothing by default
-	}
-
-	public boolean addFinalAggregate(StreamWindow<T> currentWindow) throws Exception {
-		T finalReduce = null;
-
-		if (!reduced.isEmpty()) {
-			finalReduce = reduced.get(0);
-			for (int i = 1; i < reduced.size(); i++) {
-				finalReduce = reducer.reduce(finalReduce, serializer.copy(reduced.get(i)));
-
-			}
-			if (currentReduced != null) {
-				finalReduce = reducer.reduce(finalReduce, serializer.copy(currentReduced));
-			}
-
-		} else {
-			finalReduce = currentReduced;
-		}
-
-		if (finalReduce != null) {
-			currentWindow.add(finalReduce);
-			return true;
-		} else {
-			return false;
-		}
-
-	}
-
-	public void store(T element) throws Exception {
-		addToBufferIfEligible(element);
-		afterStore();
-	}
-
-	protected void afterStore() {
-		// Do nothing by default
-	}
-
-	protected void addToBufferIfEligible(T element) throws Exception {
-		if (currentEligible(element) && currentNotEmpty()) {
-			addCurrentToBuffer(element);
-			elementsPerPreAggregate.add(elementsSinceLastPreAggregate);
-			elementsSinceLastPreAggregate = 0;
-			resetCurrent();
-		}
-		updateCurrent(element);
-
-		elementsSinceLastPreAggregate++;
-	}
-
-	protected void resetCurrent() {
-		currentReduced = null;
-	}
-
-	protected boolean currentNotEmpty() {
-		return currentReduced != null;
-	}
-
-	protected void updateCurrent(T element) throws Exception {
-		if (currentReduced == null) {
-			currentReduced = element;
-		} else {
-			currentReduced = reducer.reduce(serializer.copy(currentReduced), element);
-		}
-	}
-
-	protected void addCurrentToBuffer(T element) throws Exception {
-		reduced.add(currentReduced);
-	}
-
-	protected abstract boolean currentEligible(T next);
-
-	public void evict(int n) {
-		toRemove += n;
-
-		Integer lastPreAggregateSize = elementsPerPreAggregate.peek();
-		while (lastPreAggregateSize != null && lastPreAggregateSize <= toRemove) {
-			toRemove = max(toRemove - elementsPerPreAggregate.removeFirst(), 0);
-			removeLastReduced();
-			lastPreAggregateSize = elementsPerPreAggregate.peek();
-		}
-
-		if (lastPreAggregateSize == null) {
-			toRemove = 0;
-		}
-	}
-
-	protected void removeLastReduced() {
-		reduced.removeFirst();
-	}
-
-	public static int max(int a, int b) {
-		if (a > b) {
-			return a;
-		} else {
-			return b;
-		}
-	}
-
-	@Override
-	public abstract SlidingPreReducer<T> clone();
-
-	@Override
-	public String toString() {
-		return currentReduced.toString();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducer.java
deleted file mode 100644
index cdb4207..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducer.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-
-/**
- * Non-grouped pre-reducer for sliding time eviction policy.
- */
-public class SlidingTimeGroupedPreReducer<T> extends SlidingGroupedPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private long windowSize;
-	private long slideSize;
-	private TimestampWrapper<T> timestampWrapper;
-	private T lastStored;
-	protected long windowStartTime;
-
-	public SlidingTimeGroupedPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer,
-			KeySelector<T, ?> key, long windowSize, long slideSize,
-			TimestampWrapper<T> timestampWrapper) {
-		super(reducer, serializer, key);
-		if (windowSize > slideSize) {
-			this.windowSize = windowSize;
-			this.slideSize = slideSize;
-		} else {
-			throw new RuntimeException(
-					"Window size needs to be larger than slide size for the sliding pre-reducer");
-		}
-		this.timestampWrapper = timestampWrapper;
-		this.windowStartTime = timestampWrapper.getStartTime();
-	}
-
-	@Override
-	public void store(T element) throws Exception {
-		super.store(element);
-		lastStored = element;
-	}
-
-	@Override
-	public SlidingTimeGroupedPreReducer<T> clone() {
-		return new SlidingTimeGroupedPreReducer<T>(reducer, serializer, key, windowSize, slideSize,
-				timestampWrapper);
-	}
-
-	@Override
-	public String toString() {
-		return currentReducedMap.toString();
-	}
-
-	@Override
-	protected void afterEmit() {
-		if (lastStored != null) {
-			long lastTime = timestampWrapper.getTimestamp(lastStored);
-			if (lastTime - windowStartTime >= slideSize) {
-				windowStartTime = windowStartTime + slideSize;
-			}
-		}
-	}
-
-	@Override
-	public void evict(int n) {
-		toRemove += n;
-		Integer lastPreAggregateSize = elementsPerPreAggregate.peek();
-
-		while (lastPreAggregateSize != null && lastPreAggregateSize <= toRemove) {
-			toRemove = max(toRemove - elementsPerPreAggregate.removeFirst(), 0);
-			removeLastReduced();
-			lastPreAggregateSize = elementsPerPreAggregate.peek();
-		}
-
-		if (toRemove > 0 && lastPreAggregateSize == null) {
-			resetCurrent();
-			toRemove = 0;
-		}
-	}
-
-	@Override
-	protected boolean currentEligible(T next) {
-		return windowStartTime == timestampWrapper.getStartTime()
-				|| timestampWrapper.getTimestamp(next) - windowStartTime >= slideSize;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducer.java
deleted file mode 100644
index d84505c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducer.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-
-/**
- * Non-grouped pre-reducer for sliding time eviction policy
- * (the policies are based on time, and the slide size is smaller than the window size).
- */
-public class SlidingTimePreReducer<T> extends SlidingPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private long windowSize;
-	private long slideSize;
-	private TimestampWrapper<T> timestampWrapper;
-	private T lastStored;
-	protected long windowStartTime;
-
-	public SlidingTimePreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer,
-			long windowSize, long slideSize, TimestampWrapper<T> timestampWrapper) {
-		super(reducer, serializer);
-		if (windowSize > slideSize) {
-			this.windowSize = windowSize;
-			this.slideSize = slideSize;
-		} else {
-			throw new RuntimeException(
-					"Window size needs to be larger than slide size for the sliding pre-reducer");
-		}
-		this.timestampWrapper = timestampWrapper;
-		this.windowStartTime = timestampWrapper.getStartTime();
-	}
-
-	@Override
-	public void store(T element) throws Exception {
-		super.store(element);
-		lastStored = element;
-	}
-
-	@Override
-	public SlidingTimePreReducer<T> clone() {
-		return new SlidingTimePreReducer<T>(reducer, serializer, windowSize, slideSize,
-				timestampWrapper);
-	}
-
-	@Override
-	public String toString() {
-		return currentReduced.toString();
-	}
-
-	@Override
-	protected void afterEmit() {
-		if (lastStored != null) {
-			long lastTime = timestampWrapper.getTimestamp(lastStored);
-			if (lastTime - windowStartTime >= slideSize) {
-				windowStartTime = windowStartTime + slideSize;
-			}
-		}
-	}
-
-	@Override
-	public void evict(int n) {
-		toRemove += n;
-		Integer lastPreAggregateSize = elementsPerPreAggregate.peek();
-
-		while (lastPreAggregateSize != null && lastPreAggregateSize <= toRemove) {
-			toRemove = max(toRemove - elementsPerPreAggregate.removeFirst(), 0);
-			reduced.removeFirst();
-			lastPreAggregateSize = elementsPerPreAggregate.peek();
-		}
-
-		if (toRemove > 0 && lastPreAggregateSize == null) {
-			currentReduced = null;
-			elementsSinceLastPreAggregate = 0;
-			toRemove = 0;
-		}
-	}
-
-	@Override
-	protected boolean currentEligible(T next) {
-		return windowStartTime == timestampWrapper.getStartTime()
-				|| timestampWrapper.getTimestamp(next) - windowStartTime >= slideSize;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.java
deleted file mode 100644
index 37d3aae..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducer.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.flink.streaming.api.windowing.windowbuffer;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-/**
- * Grouped pre-reducer for tumbling eviction polciy.
- */
-public class TumblingGroupedPreReducer<T> extends WindowBuffer<T> implements PreAggregator {
-
-	private static final long serialVersionUID = 1L;
-
-	private ReduceFunction<T> reducer;
-	private KeySelector<T, ?> keySelector;
-
-	private Map<Object, T> reducedValues;
-
-	private TypeSerializer<T> serializer;
-
-	private boolean evict = true;
-
-	public TumblingGroupedPreReducer(ReduceFunction<T> reducer, KeySelector<T, ?> keySelector,
-			TypeSerializer<T> serializer) {
-		this(reducer, keySelector, serializer, true);
-	}
-
-	public TumblingGroupedPreReducer(ReduceFunction<T> reducer, KeySelector<T, ?> keySelector,
-			TypeSerializer<T> serializer, boolean evict) {
-		this.reducer = reducer;
-		this.serializer = serializer;
-		this.keySelector = keySelector;
-		this.reducedValues = new HashMap<Object, T>();
-		this.evict = evict;
-	}
-
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-
-		if (!reducedValues.isEmpty()) {
-			StreamWindow<T> currentWindow = createEmptyWindow();
-			currentWindow.addAll(reducedValues.values());
-			collector.collect(new StreamRecord<StreamWindow<T>>(currentWindow));
-		} else if (emitEmpty) {
-			collector.collect(new StreamRecord<StreamWindow<T>>(createEmptyWindow()));
-		}
-		if (evict) {
-			reducedValues.clear();
-		}
-	}
-
-	public void store(T element) throws Exception {
-		Object key = keySelector.getKey(element);
-
-		T reduced = reducedValues.get(key);
-
-		if (reduced == null) {
-			reduced = element;
-		} else {
-			reduced = reducer.reduce(serializer.copy(reduced), element);
-		}
-
-		reducedValues.put(key, reduced);
-	}
-
-	@Override
-	public void evict(int n) {
-	}
-
-	@Override
-	public TumblingGroupedPreReducer<T> clone() {
-		return new TumblingGroupedPreReducer<T>(reducer, keySelector, serializer, evict);
-	}
-
-	@Override
-	public String toString() {
-		return reducedValues.toString();
-	}
-
-	public TumblingGroupedPreReducer<T> noEvict() {
-		this.evict = false;
-		return this;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java
deleted file mode 100644
index 3a10be7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducer.java
+++ /dev/null
@@ -1,98 +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.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-/**
- * Non-grouped pre-reducer for tumbling eviction policy (the slide size is the
- * same as the window size).
- */
-public class TumblingPreReducer<T> extends WindowBuffer<T> implements PreAggregator {
-
-	private static final long serialVersionUID = 1L;
-
-	private ReduceFunction<T> reducer;
-
-	private T reduced;
-	private TypeSerializer<T> serializer;
-
-	private boolean evict = true;
-
-	public TumblingPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer) {
-		this(reducer, serializer, true);
-	}
-
-	private TumblingPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer,
-			boolean evict) {
-		this.reducer = reducer;
-		this.serializer = serializer;
-		this.evict = evict;
-	}
-
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-		if (reduced != null) {
-			StreamWindow<T> currentWindow = createEmptyWindow();
-			currentWindow.add(reduced);
-			collector.collect(new StreamRecord<StreamWindow<T>>(currentWindow));
-		} else if (emitEmpty) {
-			collector.collect(new StreamRecord<StreamWindow<T>>(createEmptyWindow()));
-		}
-
-		if (evict) {
-			reduced = null;
-		}
-	}
-
-	public void store(T element) throws Exception {
-		if (reduced == null) {
-			reduced = element;
-		} else {
-			reduced = reducer.reduce(serializer.copy(reduced), element);
-		}
-	}
-
-	public void evict(int n) {
-	}
-
-	@Override
-	public TumblingPreReducer<T> clone() {
-		return new TumblingPreReducer<T>(reducer, serializer, evict);
-	}
-
-	@Override
-	public String toString() {
-		return reduced.toString();
-	}
-
-	@Override
-	public WindowBuffer<T> emitEmpty() {
-		emitEmpty = true;
-		return this;
-	}
-
-	public TumblingPreReducer<T> noEvict() {
-		this.evict = false;
-		return this;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.java
deleted file mode 100644
index 6e87d0b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/WindowBuffer.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.flink.streaming.api.windowing.windowbuffer;
-
-import java.io.Serializable;
-
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-/**
- * Class for defining specialized buffers to store/emit window data.
- * Pre-aggregators should be implemented using this interface.
- */
-public abstract class WindowBuffer<T> implements Serializable, Cloneable {
-
-	private static final long serialVersionUID = 1L;
-
-	protected Integer nextID = 1;
-	protected boolean sequentialID = false;
-	protected boolean emitEmpty = false;
-	protected boolean emitPerGroup = false;
-
-	public abstract void store(T element) throws Exception;
-
-	public abstract void evict(int n);
-
-	public abstract void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector);
-
-	public abstract WindowBuffer<T> clone();
-
-	public WindowBuffer<T> emitEmpty() {
-		emitEmpty = true;
-		return this;
-	}
-
-	public WindowBuffer<T> sequentialID() {
-		sequentialID = true;
-		return this;
-	}
-
-	protected StreamWindow<T> createEmptyWindow() {
-		return sequentialID ? new StreamWindow<T>(nextID++) : new StreamWindow<T>();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
index 0b8482d..3a224e4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
@@ -41,17 +41,19 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.datastream.KeyedStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.datastream.WindowedDataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
 import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.graph.StreamEdge;
 import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.windowing.helper.Count;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper;
 import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
@@ -95,6 +97,8 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
 		DataStreamSink<Long> connected = dataStream1.connect(dataStream2)
 				.flatMap(new CoFlatMapFunction<Long, Long, Long>() {
+					private static final long serialVersionUID = 1L;
+
 					@Override
 					public void flatMap1(Long value, Collector<Long> out) throws Exception {
 					}
@@ -103,14 +107,17 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 					public void flatMap2(Long value, Collector<Long> out) throws Exception {
 					}
 				}).name("testCoFlatMap")
-				.window(Count.of(10))
-				.foldWindow(0L, new FoldFunction<Long, Long>() {
+				.windowAll(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(CountTrigger.of(10)))
+				.fold(0L, new FoldFunction<Long, Long>() {
+					private static final long serialVersionUID = 1L;
+
 					@Override
 					public Long fold(Long accumulator, Long value) throws Exception {
 						return null;
 					}
-				}).name("testWindowFold")
-				.flatten()
+				})
+				.name("testWindowFold")
 				.print();
 
 		//test functionality through the operator names in the execution plan
@@ -133,15 +140,15 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 	public void testPartitioning() {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		DataStream src1 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
-		DataStream src2 = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
-		ConnectedStreams connected = src1.connect(src2);
+		DataStream<Tuple2<Long, Long>> src1 = env.fromElements(new Tuple2<>(0L, 0L));
+		DataStream<Tuple2<Long, Long>> src2 = env.fromElements(new Tuple2<>(0L, 0L));
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connected = src1.connect(src2);
 
 		//Testing DataStream grouping
-		DataStream group1 = src1.keyBy(0);
-		DataStream group2 = src1.keyBy(1, 0);
-		DataStream group3 = src1.keyBy("f0");
-		DataStream group4 = src1.keyBy(new FirstSelector());
+		DataStream<Tuple2<Long, Long>> group1 = src1.keyBy(0);
+		DataStream<Tuple2<Long, Long>> group2 = src1.keyBy(1, 0);
+		DataStream<Tuple2<Long, Long>> group3 = src1.keyBy("f0");
+		DataStream<Tuple2<Long, Long>> group4 = src1.keyBy(new FirstSelector());
 
 		int id1 = createDownStreamId(group1);
 		int id2 = createDownStreamId(group2);
@@ -159,10 +166,10 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isKeyed(group4));
 
 		//Testing DataStream partitioning
-		DataStream partition1 = src1.partitionByHash(0);
-		DataStream partition2 = src1.partitionByHash(1, 0);
-		DataStream partition3 = src1.partitionByHash("f0");
-		DataStream partition4 = src1.partitionByHash(new FirstSelector());
+		DataStream<Tuple2<Long, Long>> partition1 = src1.partitionByHash(0);
+		DataStream<Tuple2<Long, Long>> partition2 = src1.partitionByHash(1, 0);
+		DataStream<Tuple2<Long, Long>> partition3 = src1.partitionByHash("f0");
+		DataStream<Tuple2<Long, Long>> partition4 = src1.partitionByHash(new FirstSelector());
 
 		int pid1 = createDownStreamId(partition1);
 		int pid2 = createDownStreamId(partition2);
@@ -187,9 +194,9 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 			}
 		};
 
-		DataStream customPartition1 = src1.partitionCustom(longPartitioner, 0);
-		DataStream customPartition3 = src1.partitionCustom(longPartitioner, "f0");
-		DataStream customPartition4 = src1.partitionCustom(longPartitioner, new FirstSelector());
+		DataStream<Tuple2<Long, Long>> customPartition1 = src1.partitionCustom(longPartitioner, 0);
+		DataStream<Tuple2<Long, Long>> customPartition3 = src1.partitionCustom(longPartitioner, "f0");
+		DataStream<Tuple2<Long, Long>> customPartition4 = src1.partitionCustom(longPartitioner, new FirstSelector());
 
 		int cid1 = createDownStreamId(customPartition1);
 		int cid2 = createDownStreamId(customPartition3);
@@ -204,19 +211,19 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertFalse(isKeyed(customPartition4));
 
 		//Testing ConnectedStreams grouping
-		ConnectedStreams connectedGroup1 = connected.keyBy(0, 0);
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup1 = connected.keyBy(0, 0);
 		Integer downStreamId1 = createDownStreamId(connectedGroup1);
 
-		ConnectedStreams connectedGroup2 = connected.keyBy(new int[]{0}, new int[]{0});
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup2 = connected.keyBy(new int[]{0}, new int[]{0});
 		Integer downStreamId2 = createDownStreamId(connectedGroup2);
 
-		ConnectedStreams connectedGroup3 = connected.keyBy("f0", "f0");
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup3 = connected.keyBy("f0", "f0");
 		Integer downStreamId3 = createDownStreamId(connectedGroup3);
 
-		ConnectedStreams connectedGroup4 = connected.keyBy(new String[]{"f0"}, new String[]{"f0"});
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup4 = connected.keyBy(new String[]{"f0"}, new String[]{"f0"});
 		Integer downStreamId4 = createDownStreamId(connectedGroup4);
 
-		ConnectedStreams connectedGroup5 = connected.keyBy(new FirstSelector(), new FirstSelector());
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup5 = connected.keyBy(new FirstSelector(), new FirstSelector());
 		Integer downStreamId5 = createDownStreamId(connectedGroup5);
 
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId1)));
@@ -241,19 +248,19 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		assertTrue(isKeyed(connectedGroup5));
 
 		//Testing ConnectedStreams partitioning
-		ConnectedStreams connectedPartition1 = connected.partitionByHash(0, 0);
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition1 = connected.partitionByHash(0, 0);
 		Integer connectDownStreamId1 = createDownStreamId(connectedPartition1);
 
-		ConnectedStreams connectedPartition2 = connected.partitionByHash(new int[]{0}, new int[]{0});
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition2 = connected.partitionByHash(new int[]{0}, new int[]{0});
 		Integer connectDownStreamId2 = createDownStreamId(connectedPartition2);
 
-		ConnectedStreams connectedPartition3 = connected.partitionByHash("f0", "f0");
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition3 = connected.partitionByHash("f0", "f0");
 		Integer connectDownStreamId3 = createDownStreamId(connectedPartition3);
 
-		ConnectedStreams connectedPartition4 = connected.partitionByHash(new String[]{"f0"}, new String[]{"f0"});
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition4 = connected.partitionByHash(new String[]{"f0"}, new String[]{"f0"});
 		Integer connectDownStreamId4 = createDownStreamId(connectedPartition4);
 
-		ConnectedStreams connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector());
+		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector());
 		Integer connectDownStreamId5 = createDownStreamId(connectedPartition5);
 
 		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
@@ -295,7 +302,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 	public void testParallelism() {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
 
-		DataStreamSource<Tuple2<Long, Long>> src = env.fromElements(new Tuple2<Long, Long>(0L, 0L));
+		DataStreamSource<Tuple2<Long, Long>> src = env.fromElements(new Tuple2<>(0L, 0L));
 		env.setParallelism(10);
 
 		SingleOutputStreamOperator<Long, ?> map = src.map(new MapFunction<Tuple2<Long, Long>, Long>() {
@@ -306,18 +313,20 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		}).name("MyMap");
 
 		DataStream<Long> windowed = map
-				.window(Count.of(10))
-				.foldWindow(0L, new FoldFunction<Long, Long>() {
+				.windowAll(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(CountTrigger.of(10)))
+				.fold(0L, new FoldFunction<Long, Long>() {
 					@Override
 					public Long fold(Long accumulator, Long value) throws Exception {
 						return null;
 					}
-				})
-				.flatten();
+				});
 
 		windowed.addSink(new NoOpSink<Long>());
 
 		DataStreamSink<Long> sink = map.addSink(new SinkFunction<Long>() {
+			private static final long serialVersionUID = 1L;
+
 			@Override
 			public void invoke(Long value) throws Exception {
 			}
@@ -343,6 +352,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 			src.setParallelism(3);
 			fail();
 		} catch (IllegalArgumentException success) {
+			// do nothing
 		}
 
 		DataStreamSource<Long> parallelSource = env.generateSequence(0, 0);
@@ -373,26 +383,33 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 			}
 		});
 
-		assertEquals(TypeExtractor.getForObject(new Tuple2<Integer, String>(0, "")), map.getType());
+		assertEquals(TypeExtractor.getForObject(new Tuple2<>(0, "")), map.getType());
 
-		WindowedDataStream<String> window = map
-				.window(Count.of(5))
-				.mapWindow(new WindowMapFunction<Tuple2<Integer, String>, String>() {
+		DataStream<String> window = map
+				.windowAll(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(CountTrigger.of(5)))
+				.apply(new AllWindowFunction<Tuple2<Integer, String>, String, GlobalWindow>() {
 					@Override
-					public void mapWindow(Iterable<Tuple2<Integer, String>> values, Collector<String> out) throws Exception {
+					public void apply(GlobalWindow window,
+							Iterable<Tuple2<Integer, String>> values,
+							Collector<String> out) throws Exception {
+
 					}
 				});
 
 		assertEquals(TypeExtractor.getForClass(String.class), window.getType());
 
 		DataStream<CustomPOJO> flatten = window
-				.foldWindow(new CustomPOJO(), new FoldFunction<String, CustomPOJO>() {
+				.windowAll(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(CountTrigger.of(5)))
+				.fold(new CustomPOJO(), new FoldFunction<String, CustomPOJO>() {
+					private static final long serialVersionUID = 1L;
+
 					@Override
 					public CustomPOJO fold(CustomPOJO accumulator, String value) throws Exception {
 						return null;
 					}
-				})
-				.flatten();
+				});
 
 		assertEquals(TypeExtractor.getForClass(CustomPOJO.class), flatten.getType());
 	}
@@ -415,6 +432,8 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
 
 		FlatMapFunction<Long, Integer> flatMapFunction = new FlatMapFunction<Long, Integer>() {
+			private static final long serialVersionUID = 1L;
+
 			@Override
 			public void flatMap(Long value, Collector<Integer> out) throws Exception {
 			}
@@ -430,8 +449,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 			}
 		};
 
-		DataStream<Integer> unionFilter = map
-				.union(flatMap)
+		DataStream<Integer> unionFilter = map.union(flatMap)
 				.filter(filterFunction);
 
 		unionFilter.addSink(new NoOpSink<Integer>());
@@ -471,6 +489,8 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 
 		ConnectedStreams<Integer, Integer> connect = map.connect(flatMap);
 		CoMapFunction<Integer, Integer, String> coMapper = new CoMapFunction<Integer, Integer, String>() {
+			private static final long serialVersionUID = 1L;
+
 			@Override
 			public String map1(Integer value) {
 				return null;
@@ -597,16 +617,19 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		return operator.getUserFunction();
 	}
 
-	private static Integer createDownStreamId(DataStream dataStream) {
+	private static Integer createDownStreamId(DataStream<?> dataStream) {
 		return dataStream.print().getTransformation().getId();
 	}
 
-	private static boolean isKeyed(DataStream dataStream) {
+	private static boolean isKeyed(DataStream<?> dataStream) {
 		return dataStream instanceof KeyedStream;
 	}
 
+	@SuppressWarnings("rawtypes,unchecked")
 	private static Integer createDownStreamId(ConnectedStreams dataStream) {
-		SingleOutputStreamOperator coMap = dataStream.map(new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() {
+		SingleOutputStreamOperator<?, ?> coMap = dataStream.map(new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() {
+			private static final long serialVersionUID = 1L;
+
 			@Override
 			public Object map1(Tuple2<Long, Long> value) {
 				return null;
@@ -621,7 +644,7 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 		return coMap.getId();
 	}
 
-	private static boolean isKeyed(ConnectedStreams dataStream) {
+	private static boolean isKeyed(ConnectedStreams<?, ?> dataStream) {
 		return (dataStream.getFirstInput() instanceof KeyedStream && dataStream.getSecondInput() instanceof KeyedStream);
 	}
 
@@ -634,6 +657,8 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
 	}
 
 	private static class FirstSelector implements KeySelector<Tuple2<Long, Long>, Long> {
+		private static final long serialVersionUID = 1L;
+
 		@Override
 		public Long getKey(Tuple2<Long, Long> value) throws Exception {
 			return value.f0;

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
index 5e46508..2775299 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
@@ -33,15 +33,18 @@ import org.apache.flink.streaming.api.datastream.IterativeStream;
 import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
 import org.apache.flink.streaming.api.datastream.SplitStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
 import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.windowing.helper.Count;
-import org.apache.flink.streaming.api.windowing.helper.Delta;
-import org.apache.flink.streaming.api.windowing.helper.Time;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.After;
@@ -59,6 +62,7 @@ import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 @SuppressWarnings("serial")
 public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
@@ -117,8 +121,8 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 		IterativeStream<Tuple2<Long, Tuple2<String, Long>>> it = sourceStream1.map(new MapFunction<Tuple2<Long, Tuple2<String, Long>>,Tuple2<Long, Tuple2<String, Long>>>(){
 
-					Tuple2<Long, Tuple2<String, Long>> result = new Tuple2<Long, Tuple2<String, Long>>(
-							0L, new Tuple2<String, Long>("", 0L));
+					Tuple2<Long, Tuple2<String, Long>> result = new Tuple2<>(
+							0L, new Tuple2<>("", 0L));
 
 					@Override
 					public Tuple2<Long, Tuple2<String, Long>> map(
@@ -167,38 +171,38 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 				"peach-d\n" + "peach-d\n";
 
 		List<Tuple5<Integer, String, Character, Double, Boolean>> input = Arrays.asList(
-				new Tuple5<Integer, String, Character, Double, Boolean>(1, "apple", 'j', 0.1, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(1, "peach", 'b', 0.8, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(1, "orange", 'c', 0.7, true),
-				new Tuple5<Integer, String, Character, Double, Boolean>(2, "apple", 'd', 0.5, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(2, "peach", 'j', 0.6, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(3, "orange", 'b', 0.2, true),
-				new Tuple5<Integer, String, Character, Double, Boolean>(6, "apple", 'c', 0.1, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(7, "peach", 'd', 0.4, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(8, "orange", 'j', 0.2, true),
-				new Tuple5<Integer, String, Character, Double, Boolean>(10, "apple", 'b', 0.1, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(10, "peach", 'c', 0.5, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(11, "orange", 'd', 0.3, true),
-				new Tuple5<Integer, String, Character, Double, Boolean>(11, "apple", 'j', 0.3, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(12, "peach", 'b', 0.9, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(13, "orange", 'c', 0.7, true),
-				new Tuple5<Integer, String, Character, Double, Boolean>(15, "apple", 'd', 0.2, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(16, "peach", 'j', 0.8, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(16, "orange", 'b', 0.8, true),
-				new Tuple5<Integer, String, Character, Double, Boolean>(16, "apple", 'c', 0.1, false),
-				new Tuple5<Integer, String, Character, Double, Boolean>(17, "peach", 'd', 1.0, true));
+				new Tuple5<>(1, "apple", 'j', 0.1, false),
+				new Tuple5<>(1, "peach", 'b', 0.8, false),
+				new Tuple5<>(1, "orange", 'c', 0.7, true),
+				new Tuple5<>(2, "apple", 'd', 0.5, false),
+				new Tuple5<>(2, "peach", 'j', 0.6, false),
+				new Tuple5<>(3, "orange", 'b', 0.2, true),
+				new Tuple5<>(6, "apple", 'c', 0.1, false),
+				new Tuple5<>(7, "peach", 'd', 0.4, false),
+				new Tuple5<>(8, "orange", 'j', 0.2, true),
+				new Tuple5<>(10, "apple", 'b', 0.1, false),
+				new Tuple5<>(10, "peach", 'c', 0.5, false),
+				new Tuple5<>(11, "orange", 'd', 0.3, true),
+				new Tuple5<>(11, "apple", 'j', 0.3, false),
+				new Tuple5<>(12, "peach", 'b', 0.9, false),
+				new Tuple5<>(13, "orange", 'c', 0.7, true),
+				new Tuple5<>(15, "apple", 'd', 0.2, false),
+				new Tuple5<>(16, "peach", 'j', 0.8, false),
+				new Tuple5<>(16, "orange", 'b', 0.8, true),
+				new Tuple5<>(16, "apple", 'c', 0.1, false),
+				new Tuple5<>(17, "peach", 'd', 1.0, true));
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().enableTimestamps();
 
 		SingleOutputStreamOperator<Tuple5<Integer, String, Character, Double, Boolean>, DataStreamSource<Tuple5<Integer, String, Character, Double, Boolean>>> sourceStream21 = env.fromCollection(input);
 		DataStream<OuterPojo> sourceStream22 = env.addSource(new PojoSource());
 
 		sourceStream21
+				.extractTimestamp(new MyTimestampExtractor())
 				.keyBy(2, 2)
-				.window(Time.of(10, new MyTimestamp(), 0))
-				.every(Time.of(4, new MyTimestamp(), 0))
+				.timeWindow(Time.of(10, TimeUnit.MILLISECONDS), Time.of(4, TimeUnit.MILLISECONDS))
 				.maxBy(3)
-				.flatten()
 				.map(new MyMapFunction2())
 				.flatMap(new MyFlatMapFunction())
 				.connect(sourceStream22)
@@ -244,11 +248,13 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 		DataStream<Long> sourceStream32 = env.generateSequence(10001, 20000);
 
 		sourceStream31.filter(new PrimeFilterFunction())
-				.window(Count.of(100))
-				.max(0).flatten()
+				.windowAll(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(CountTrigger.of(100)))
+				.max(0)
 				.union(sourceStream32.filter(new PrimeFilterFunction())
-						.window(Count.of(100))
-						.max(0).flatten())
+						.windowAll(GlobalWindows.create())
+						.trigger(PurgingTrigger.of(CountTrigger.of(100)))
+						.max(0))
 				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
 
 		sourceStream31.flatMap(new DivisorsFlatMapFunction())
@@ -257,11 +263,13 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 			@Override
 			public Tuple2<Long, Integer> map(Long value) throws Exception {
-				return new Tuple2<Long, Integer>(value, 1);
+				return new Tuple2<>(value, 1);
 			}
 		})
 				.keyBy(0)
-				.window(Count.of(10000)).sum(1).flatten()
+				.window(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(CountTrigger.of(10_000)))
+				.sum(1)
 				.filter(new FilterFunction<Tuple2<Long, Integer>>() {
 
 					@Override
@@ -275,6 +283,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 	}
 
 	@Test
+	@Ignore
 	public void complexIntegrationTest4() throws Exception {
 		//Testing mapping and delta-policy windowing with custom class
 
@@ -290,13 +299,14 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 				"((499,587),90)\n" + "((516,606),93)\n" + "((517,609),94)\n" + "((534,628),97)\n" + "((535,631),98)";
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(1);
 
 		env.addSource(new RectangleSource())
 				.global()
 				.map(new RectangleMapFunction())
-				.window(Delta.of(0.0, new MyDelta(), new Tuple2<Rectangle, Integer>(new Rectangle(100, 100), 0)))
-				.mapWindow(new MyWindowMapFunction())
-				.flatten()
+				.windowAll(GlobalWindows.create())
+				.trigger(PurgingTrigger.of(DeltaTrigger.of(0.0, new MyDelta())))
+				.apply(new MyWindowMapFunction())
 				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
 
 		env.execute();
@@ -361,6 +371,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 
 	@Test
+	@Ignore
 	public void complexIntegrationTest6() throws Exception {
 		//Testing java collections and date-time types
 
@@ -376,88 +387,89 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 		SimpleDateFormat ft = new SimpleDateFormat("dd-MM-yyyy");
 
-		ArrayList<Tuple2<Date, HashMap<Character, Integer>>> sales = new ArrayList<Tuple2<Date, HashMap<Character,
-				Integer>>>();
-		HashMap<Character, Integer> sale1 = new HashMap<Character, Integer>();
+		ArrayList<Tuple2<Date, HashMap<Character, Integer>>> sales = new ArrayList<>();
+		HashMap<Character, Integer> sale1 = new HashMap<>();
 		sale1.put('a', 2);
 		sale1.put('c', 2);
 		sale1.put('d', 1);
 		sale1.put('f', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("03-06-2014"), sale1));
+		sales.add(new Tuple2<>(ft.parse("03-06-2014"), sale1));
 
-		HashMap<Character, Integer> sale2 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale2 = new HashMap<>();
 		sale2.put('a', 1);
 		sale2.put('b', 2);
 		sale2.put('d', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("10-06-2014"), sale2));
+		sales.add(new Tuple2<>(ft.parse("10-06-2014"), sale2));
 
-		HashMap<Character, Integer> sale3 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale3 = new HashMap<>();
 		sale3.put('a', 3);
 		sale3.put('b', 1);
 		sale3.put('c', 2);
 		sale3.put('f', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("29-06-2014"), sale3));
+		sales.add(new Tuple2<>(ft.parse("29-06-2014"), sale3));
 
-		HashMap<Character, Integer> sale4 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale4 = new HashMap<>();
 		sale4.put('a', 1);
 		sale4.put('d', 1);
 		sale4.put('e', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("15-07-2014"), sale4));
+		sales.add(new Tuple2<>(ft.parse("15-07-2014"), sale4));
 
-		HashMap<Character, Integer> sale5 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale5 = new HashMap<>();
 		sale5.put('b', 2);
 		sale5.put('c', 3);
 		sale5.put('f', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("24-07-2014"), sale5));
+		sales.add(new Tuple2<>(ft.parse("24-07-2014"), sale5));
 
-		HashMap<Character, Integer> sale6 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale6 = new HashMap<>();
 		sale6.put('a', 4);
 		sale6.put('b', 2);
 		sale6.put('c', 2);
 		sale6.put('e', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("17-08-2014"), sale6));
+		sales.add(new Tuple2<>(ft.parse("17-08-2014"), sale6));
 
-		HashMap<Character, Integer> sale7 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale7 = new HashMap<>();
 		sale7.put('a', 2);
 		sale7.put('b', 2);
 		sale7.put('c', 3);
 		sale7.put('d', 1);
 		sale7.put('e', 1);
 		sale7.put('f', 2);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("27-08-2014"), sale7));
+		sales.add(new Tuple2<>(ft.parse("27-08-2014"), sale7));
 
-		HashMap<Character, Integer> sale8 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale8 = new HashMap<>();
 		sale8.put('a', 3);
 		sale8.put('b', 1);
 		sale8.put('c', 3);
 		sale8.put('d', 2);
 		sale8.put('f', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("16-09-2014"), sale8));
+		sales.add(new Tuple2<>(ft.parse("16-09-2014"), sale8));
 
-		HashMap<Character, Integer> sale9 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale9 = new HashMap<>();
 		sale9.put('a', 1);
 		sale9.put('b', 3);
 		sale9.put('c', 4);
 		sale9.put('d', 1);
 		sale9.put('e', 1);
 		sale9.put('f', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("25-09-2014"), sale9));
+		sales.add(new Tuple2<>(ft.parse("25-09-2014"), sale9));
 
-		HashMap<Character, Integer> sale10 = new HashMap<Character, Integer>();
+		HashMap<Character, Integer> sale10 = new HashMap<>();
 		sale10.put('a', 3);
 		sale10.put('b', 2);
 		sale10.put('c', 3);
 		sale10.put('d', 2);
 		sale10.put('e', 1);
 		sale10.put('f', 1);
-		sales.add(new Tuple2<Date, HashMap<Character, Integer>>(ft.parse("01-10-2014"), sale10));
+		sales.add(new Tuple2<>(ft.parse("01-10-2014"), sale10));
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.getConfig().enableTimestamps();
 
 		DataStream<Tuple2<Date, HashMap<Character, Integer>>> sourceStream6 = env.fromCollection(sales);
-		sourceStream6.window(Time.of(1, new Timestamp6()))
-				.reduceWindow(new SalesReduceFunction())
-				.flatten()
+		sourceStream6
+				.extractTimestamp(new Timestamp6())
+				.timeWindowAll(Time.of(1, TimeUnit.MILLISECONDS))
+				.reduce(new SalesReduceFunction())
 				.flatMap(new FlatMapFunction6())
 				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
 
@@ -478,7 +490,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 		@Override
 		public Tuple4<Integer, String, Double, Boolean> map(Tuple5<Integer, String, Character, Double,
 				Boolean> value) throws Exception {
-			return new Tuple4<Integer, String, Double, Boolean>(value.f0, value.f1 + "-" + value.f2,
+			return new Tuple4<>(value.f0, value.f1 + "-" + value.f2,
 					value.f3, value.f4);
 		}
 
@@ -509,7 +521,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 		@Override
 		public void run(SourceContext<Tuple2<Long, Tuple2<String, Long>>> ctx) throws Exception {
 			for (int i = 0; i < 20; i++) {
-				Tuple2<Long, Tuple2<String, Long>> result = new Tuple2<Long, Tuple2<String, Long>>(1L, new Tuple2<String, Long>("a", 1L));
+				Tuple2<Long, Tuple2<String, Long>> result = new Tuple2<>(1L, new Tuple2<>("a", 1L));
 				ctx.collect(result);
 			}
 		}
@@ -526,17 +538,28 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 		@Override
 		public Tuple2<Long, Tuple2<String, Long>> map(Tuple2<Long, Tuple2<String, Long>> value) throws Exception {
-			return new Tuple2<Long, Tuple2<String, Long>>(value.f0 + 1, value.f1);
+			return new Tuple2<>(value.f0 + 1, value.f1);
 		}
 	}
 
-	private static class MyTimestamp implements Timestamp<Tuple5<Integer, String, Character, Double, Boolean>> {
+	private static class MyTimestampExtractor implements TimestampExtractor<Tuple5<Integer, String, Character, Double, Boolean>> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public long getTimestamp(Tuple5<Integer, String, Character, Double, Boolean> value) {
+		public long extractTimestamp(Tuple5<Integer, String, Character, Double, Boolean> value, long currentTimestamp) {
 			return (long) value.f0;
 		}
+
+		@Override
+		public long emitWatermark(Tuple5<Integer, String, Character, Double, Boolean> value,
+				long currentTimestamp) {
+			return (long) value.f0 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
 	}
 
 	private static class MyFlatMapFunction implements FlatMapFunction<Tuple4<Integer, String, Double,
@@ -573,7 +596,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 		@Override
 		public Iterable<String> select(Tuple2<Long, Tuple2<String, Long>> value) {
-			List<String> output = new ArrayList<String>();
+			List<String> output = new ArrayList<>();
 			if (value.f0 == 10) {
 				output.add("iterate");
 				output.add("firstOutput");
@@ -627,6 +650,8 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 		@Override
 		public void run(SourceContext<Rectangle> ctx) throws Exception {
+			// emit once as the initializer of the delta trigger
+			ctx.collect(rectangle);
 			for (int i = 0; i < 100; i++) {
 				ctx.collect(rectangle);
 				rectangle = rectangle.next();
@@ -644,16 +669,15 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 
 		@Override
 		public Tuple2<Rectangle, Integer> map(Rectangle value) throws Exception {
-			return new Tuple2<Rectangle, Integer>(value, counter++);
+			return new Tuple2<>(value, counter++);
 		}
 	}
 
-	private static class MyWindowMapFunction implements WindowMapFunction<Tuple2<Rectangle, Integer>,
-			Tuple2<Rectangle, Integer>> {
+	private static class MyWindowMapFunction implements AllWindowFunction<Tuple2<Rectangle, Integer>, Tuple2<Rectangle, Integer>, GlobalWindow> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public void mapWindow(Iterable<Tuple2<Rectangle, Integer>> values, Collector<Tuple2<Rectangle,
+		public void apply(GlobalWindow window, Iterable<Tuple2<Rectangle, Integer>> values, Collector<Tuple2<Rectangle,
 				Integer>> out) throws Exception {
 			out.collect(values.iterator().next());
 		}
@@ -670,14 +694,28 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 		}
 	}
 
-	private static class Timestamp6 implements Timestamp<Tuple2<Date, HashMap<Character, Integer>>> {
+	private static class Timestamp6 implements TimestampExtractor<Tuple2<Date, HashMap<Character, Integer>>> {
 
 		@Override
-		public long getTimestamp(Tuple2<Date, HashMap<Character, Integer>> value) {
+		public long extractTimestamp(Tuple2<Date, HashMap<Character, Integer>> value,
+				long currentTimestamp) {
 			Calendar cal = Calendar.getInstance();
 			cal.setTime(value.f0);
 			return 12 * (cal.get(Calendar.YEAR)) + cal.get(Calendar.MONTH);
 		}
+
+		@Override
+		public long emitWatermark(Tuple2<Date, HashMap<Character, Integer>> value,
+				long currentTimestamp) {
+			Calendar cal = Calendar.getInstance();
+			cal.setTime(value.f0);
+			return 12 * (cal.get(Calendar.YEAR)) + cal.get(Calendar.MONTH) - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return 0;
+		}
 	}
 
 	private static class SalesReduceFunction implements ReduceFunction<Tuple2<Date, HashMap<Character, Integer>>> {
@@ -697,7 +735,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 				}
 				map1.put(key, volume1 + volume2);
 			}
-			return new Tuple2<Date, HashMap<Character, Integer>>(value2.f0, map1);
+			return new Tuple2<>(value2.f0, map1);
 		}
 	}
 
@@ -710,9 +748,9 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 			Calendar cal = Calendar.getInstance();
 			cal.setTime(value.f0);
 			for (Character key : value.f1.keySet()) {
-				out.collect(new Tuple2<Integer, Tuple2<Character, Integer>>(cal.get(Calendar.MONTH)
+				out.collect(new Tuple2<>(cal.get(Calendar.MONTH)
 						+ 1,
-						new Tuple2<Character, Integer>(key, value.f1.get(key))));
+						new Tuple2<>(key, value.f1.get(key))));
 			}
 		}
 	}
@@ -722,7 +760,7 @@ public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
 		@Override
 		public ArrayList<Character> map(Tuple2<Date, HashMap<Character, Integer>> value)
 				throws Exception {
-			ArrayList<Character> list = new ArrayList<Character>();
+			ArrayList<Character> list = new ArrayList<>();
 			for (Character ch : value.f1.keySet()) {
 				for (int i = 0; i < value.f1.get(ch); i++) {
 					list.add(ch);

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTupleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTupleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTupleTest.java
new file mode 100644
index 0000000..c98a659
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTupleTest.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.flink.streaming.api.functions.windowing.delta.extractor;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple10;
+import org.apache.flink.api.java.tuple.Tuple11;
+import org.apache.flink.api.java.tuple.Tuple12;
+import org.apache.flink.api.java.tuple.Tuple13;
+import org.apache.flink.api.java.tuple.Tuple14;
+import org.apache.flink.api.java.tuple.Tuple15;
+import org.apache.flink.api.java.tuple.Tuple16;
+import org.apache.flink.api.java.tuple.Tuple17;
+import org.apache.flink.api.java.tuple.Tuple18;
+import org.apache.flink.api.java.tuple.Tuple19;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple20;
+import org.apache.flink.api.java.tuple.Tuple21;
+import org.apache.flink.api.java.tuple.Tuple22;
+import org.apache.flink.api.java.tuple.Tuple23;
+import org.apache.flink.api.java.tuple.Tuple24;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.tuple.Tuple8;
+import org.apache.flink.api.java.tuple.Tuple9;
+import org.apache.flink.streaming.api.functions.windowing.delta.extractor.ArrayFromTuple;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ArrayFromTupleTest {
+
+	private String[] testStrings;
+
+	@Before
+	public void init() {
+		testStrings = new String[Tuple.MAX_ARITY];
+		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
+			testStrings[i] = Integer.toString(i);
+		}
+	}
+
+	@Test
+	public void testConvertFromTupleToArray() throws InstantiationException, IllegalAccessException {
+		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
+			Tuple currentTuple = (Tuple) CLASSES[i].newInstance();
+			String[] currentArray = new String[i + 1];
+			for (int j = 0; j <= i; j++) {
+				currentTuple.setField(testStrings[j], j);
+				currentArray[j] = testStrings[j];
+			}
+			arrayEqualityCheck(currentArray, new ArrayFromTuple().extract(currentTuple));
+		}
+	}
+
+	@Test
+	public void testUserSpecifiedOrder() throws InstantiationException, IllegalAccessException {
+		Tuple currentTuple = (Tuple) CLASSES[Tuple.MAX_ARITY - 1].newInstance();
+		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
+			currentTuple.setField(testStrings[i], i);
+		}
+
+		String[] expected = { testStrings[5], testStrings[3], testStrings[6], testStrings[7],
+				testStrings[0] };
+		arrayEqualityCheck(expected, new ArrayFromTuple(5, 3, 6, 7, 0).extract(currentTuple));
+
+		String[] expected2 = { testStrings[0], testStrings[Tuple.MAX_ARITY - 1] };
+		arrayEqualityCheck(expected2,
+				new ArrayFromTuple(0, Tuple.MAX_ARITY - 1).extract(currentTuple));
+
+		String[] expected3 = { testStrings[Tuple.MAX_ARITY - 1], testStrings[0] };
+		arrayEqualityCheck(expected3,
+				new ArrayFromTuple(Tuple.MAX_ARITY - 1, 0).extract(currentTuple));
+
+		String[] expected4 = { testStrings[13], testStrings[4], testStrings[5], testStrings[4],
+				testStrings[2], testStrings[8], testStrings[6], testStrings[2], testStrings[8],
+				testStrings[3], testStrings[5], testStrings[2], testStrings[16], testStrings[4],
+				testStrings[3], testStrings[2], testStrings[6], testStrings[4], testStrings[7],
+				testStrings[4], testStrings[2], testStrings[8], testStrings[7], testStrings[2] };
+		arrayEqualityCheck(expected4, new ArrayFromTuple(13, 4, 5, 4, 2, 8, 6, 2, 8, 3, 5, 2, 16,
+				4, 3, 2, 6, 4, 7, 4, 2, 8, 7, 2).extract(currentTuple));
+	}
+
+	private void arrayEqualityCheck(Object[] array1, Object[] array2) {
+		assertEquals("The result arrays must have the same length", array1.length, array2.length);
+		for (int i = 0; i < array1.length; i++) {
+			assertEquals("Unequal fields at position " + i, array1[i], array2[i]);
+		}
+	}
+
+	private static final Class<?>[] CLASSES = new Class<?>[] { Tuple1.class, Tuple2.class,
+			Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class,
+			Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class,
+			Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class,
+			Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class,
+			Tuple24.class, Tuple25.class };
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtractTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtractTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtractTest.java
new file mode 100644
index 0000000..3b098c3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtractTest.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ConcatenatedExtractTest {
+
+	private String[] testStringArray1 = { "1", "2", "3" };
+	private int[] testIntArray1 = { 1, 2, 3 };
+	private String[] testStringArray2 = { "4", "5", "6" };
+	private int[] testIntArray2 = { 4, 5, 6 };
+	private String[] testStringArray3 = { "7", "8", "9" };
+	private int[] testIntArray3 = { 7, 8, 9 };
+	private Tuple2<String[], int[]>[] testTuple2Array;
+	private Tuple2<String[], int[]> testTuple2;
+	private Tuple2<Tuple2<String[], int[]>, Tuple2<String[], int[]>[]> testData;
+
+	@SuppressWarnings("unchecked")
+	@Before
+	public void setupData() {
+		testTuple2Array = new Tuple2[2];
+		testTuple2Array[0] = new Tuple2<String[], int[]>(testStringArray1, testIntArray2);
+		testTuple2Array[1] = new Tuple2<String[], int[]>(testStringArray2, testIntArray1);
+
+		testTuple2 = new Tuple2<String[], int[]>(testStringArray3, testIntArray3);
+
+		testData = new Tuple2<Tuple2<String[], int[]>, Tuple2<String[], int[]>[]>(testTuple2,
+				testTuple2Array);
+	}
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	@Test
+	public void test1() {
+		Extractor ext = new ConcatenatedExtract(new FieldFromTuple(0), new FieldFromTuple(1))
+				.add(new FieldsFromArray(Integer.class, 2, 1, 0));
+		int[] expected = { testIntArray3[2], testIntArray3[1], testIntArray3[0] };
+		assertEquals(new Integer(expected[0]), ((Integer[]) ext.extract(testData))[0]);
+		assertEquals(new Integer(expected[1]), ((Integer[]) ext.extract(testData))[1]);
+		assertEquals(new Integer(expected[2]), ((Integer[]) ext.extract(testData))[2]);
+	}
+
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	@Test
+	public void test2() {
+		Extractor ext = new ConcatenatedExtract(new FieldFromTuple(1), // Tuple2<String[],int[]>[]
+				new FieldsFromArray(Tuple2.class, 1)) // Tuple2<String[],int[]>[]
+				.add(new FieldFromArray(0)) // Tuple2<String[],int[]>
+				.add(new ArrayFromTuple(0)) // Object[] (Containing String[])
+				.add(new FieldFromArray(0)) // String[]
+				.add(new FieldFromArray(1)); // String
+
+		String expected2 = testStringArray2[1];
+		assertEquals(expected2, ext.extract(testData));
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArrayTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArrayTest.java
new file mode 100644
index 0000000..d274f4e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArrayTest.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.flink.streaming.api.functions.windowing.delta.extractor;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class FieldFromArrayTest {
+
+	String[] testStringArray = { "0", "1", "2", "3", "4" };
+	Integer[] testIntegerArray = { 10, 11, 12, 13, 14 };
+	int[] testIntArray = { 20, 21, 22, 23, 24 };
+
+	@Test
+	public void testStringArray() {
+		for (int i = 0; i < this.testStringArray.length; i++) {
+			assertEquals(this.testStringArray[i],
+					new FieldFromArray<String>(i).extract(testStringArray));
+		}
+	}
+
+	@Test
+	public void testIntegerArray() {
+		for (int i = 0; i < this.testIntegerArray.length; i++) {
+			assertEquals(this.testIntegerArray[i],
+					new FieldFromArray<String>(i).extract(testIntegerArray));
+		}
+	}
+
+	@Test
+	public void testIntArray() {
+		for (int i = 0; i < this.testIntArray.length; i++) {
+			assertEquals(new Integer(this.testIntArray[i]),
+					new FieldFromArray<Integer>(i).extract(testIntArray));
+		}
+	}
+
+}


[03/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtractTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtractTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtractTest.java
deleted file mode 100644
index e99de38..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtractTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.extractor;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.junit.Before;
-import org.junit.Test;
-
-public class ConcatenatedExtractTest {
-
-	private String[] testStringArray1 = { "1", "2", "3" };
-	private int[] testIntArray1 = { 1, 2, 3 };
-	private String[] testStringArray2 = { "4", "5", "6" };
-	private int[] testIntArray2 = { 4, 5, 6 };
-	private String[] testStringArray3 = { "7", "8", "9" };
-	private int[] testIntArray3 = { 7, 8, 9 };
-	private Tuple2<String[], int[]>[] testTuple2Array;
-	private Tuple2<String[], int[]> testTuple2;
-	private Tuple2<Tuple2<String[], int[]>, Tuple2<String[], int[]>[]> testData;
-
-	@SuppressWarnings("unchecked")
-	@Before
-	public void setupData() {
-		testTuple2Array = new Tuple2[2];
-		testTuple2Array[0] = new Tuple2<String[], int[]>(testStringArray1, testIntArray2);
-		testTuple2Array[1] = new Tuple2<String[], int[]>(testStringArray2, testIntArray1);
-
-		testTuple2 = new Tuple2<String[], int[]>(testStringArray3, testIntArray3);
-
-		testData = new Tuple2<Tuple2<String[], int[]>, Tuple2<String[], int[]>[]>(testTuple2,
-				testTuple2Array);
-	}
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void test1() {
-		Extractor ext = new ConcatenatedExtract(new FieldFromTuple(0), new FieldFromTuple(1))
-				.add(new FieldsFromArray(Integer.class, 2, 1, 0));
-		int[] expected = { testIntArray3[2], testIntArray3[1], testIntArray3[0] };
-		assertEquals(new Integer(expected[0]), ((Integer[]) ext.extract(testData))[0]);
-		assertEquals(new Integer(expected[1]), ((Integer[]) ext.extract(testData))[1]);
-		assertEquals(new Integer(expected[2]), ((Integer[]) ext.extract(testData))[2]);
-	}
-
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	@Test
-	public void test2() {
-		Extractor ext = new ConcatenatedExtract(new FieldFromTuple(1), // Tuple2<String[],int[]>[]
-				new FieldsFromArray(Tuple2.class, 1)) // Tuple2<String[],int[]>[]
-				.add(new FieldFromArray(0)) // Tuple2<String[],int[]>
-				.add(new ArrayFromTuple(0)) // Object[] (Containing String[])
-				.add(new FieldFromArray(0)) // String[]
-				.add(new FieldFromArray(1)); // String
-
-		String expected2 = testStringArray2[1];
-		assertEquals(expected2, ext.extract(testData));
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArrayTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArrayTest.java
deleted file mode 100644
index 2d4dbcf..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArrayTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.extractor;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.streaming.api.windowing.extractor.FieldFromArray;
-import org.junit.Test;
-
-public class FieldFromArrayTest {
-
-	String[] testStringArray = { "0", "1", "2", "3", "4" };
-	Integer[] testIntegerArray = { 10, 11, 12, 13, 14 };
-	int[] testIntArray = { 20, 21, 22, 23, 24 };
-
-	@Test
-	public void testStringArray() {
-		for (int i = 0; i < this.testStringArray.length; i++) {
-			assertEquals(this.testStringArray[i],
-					new FieldFromArray<String>(i).extract(testStringArray));
-		}
-	}
-
-	@Test
-	public void testIntegerArray() {
-		for (int i = 0; i < this.testIntegerArray.length; i++) {
-			assertEquals(this.testIntegerArray[i],
-					new FieldFromArray<String>(i).extract(testIntegerArray));
-		}
-	}
-
-	@Test
-	public void testIntArray() {
-		for (int i = 0; i < this.testIntArray.length; i++) {
-			assertEquals(new Integer(this.testIntArray[i]),
-					new FieldFromArray<Integer>(i).extract(testIntArray));
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTupleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTupleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTupleTest.java
deleted file mode 100644
index 528611a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTupleTest.java
+++ /dev/null
@@ -1,84 +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.flink.streaming.api.windowing.extractor;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple10;
-import org.apache.flink.api.java.tuple.Tuple11;
-import org.apache.flink.api.java.tuple.Tuple12;
-import org.apache.flink.api.java.tuple.Tuple13;
-import org.apache.flink.api.java.tuple.Tuple14;
-import org.apache.flink.api.java.tuple.Tuple15;
-import org.apache.flink.api.java.tuple.Tuple16;
-import org.apache.flink.api.java.tuple.Tuple17;
-import org.apache.flink.api.java.tuple.Tuple18;
-import org.apache.flink.api.java.tuple.Tuple19;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple20;
-import org.apache.flink.api.java.tuple.Tuple21;
-import org.apache.flink.api.java.tuple.Tuple22;
-import org.apache.flink.api.java.tuple.Tuple23;
-import org.apache.flink.api.java.tuple.Tuple24;
-import org.apache.flink.api.java.tuple.Tuple25;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple6;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.tuple.Tuple8;
-import org.apache.flink.api.java.tuple.Tuple9;
-import org.apache.flink.streaming.api.windowing.extractor.FieldFromTuple;
-import org.junit.Before;
-import org.junit.Test;
-
-public class FieldFromTupleTest {
-
-	private String[] testStrings;
-
-	@Before
-	public void init() {
-		testStrings = new String[Tuple.MAX_ARITY];
-		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
-			testStrings[i] = Integer.toString(i);
-		}
-	}
-
-	@Test
-	public void testSingleFieldExtraction() throws InstantiationException, IllegalAccessException {
-		// extract single fields
-		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
-			Tuple current = (Tuple) CLASSES[i].newInstance();
-			for (int j = 0; j < i; j++) {
-				current.setField(testStrings[j], j);
-			}
-			for (int j = 0; j < i; j++) {
-				assertEquals(testStrings[j], new FieldFromTuple<String>(j).extract(current));
-			}
-		}
-	}
-
-	private static final Class<?>[] CLASSES = new Class<?>[] { Tuple1.class, Tuple2.class,
-			Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class,
-			Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class,
-			Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class,
-			Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class,
-			Tuple24.class, Tuple25.class };
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArrayTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArrayTest.java
deleted file mode 100644
index 3139aa5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArrayTest.java
+++ /dev/null
@@ -1,108 +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.flink.streaming.api.windowing.extractor;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.flink.streaming.api.windowing.extractor.FieldsFromArray;
-import org.junit.Test;
-
-public class FieldsFromArrayTest {
-
-	String[] testStringArray = { "0", "1", "2", "3", "4" };
-	Integer[] testIntegerArray = { 10, 11, 12, 13, 14 };
-	int[] testIntArray = { 20, 21, 22, 23, 24 };
-
-	@Test
-	public void testStringArray() {
-		// check single field extraction
-		for (int i = 0; i < testStringArray.length; i++) {
-			String[] tmp = { testStringArray[i] };
-			arrayEqualityCheck(tmp,
-					new FieldsFromArray<String>(String.class, i).extract(testStringArray));
-		}
-
-		// check reverse order
-		String[] reverseOrder = new String[testStringArray.length];
-		for (int i = 0; i < testStringArray.length; i++) {
-			reverseOrder[i] = testStringArray[testStringArray.length - i - 1];
-		}
-		arrayEqualityCheck(reverseOrder,
-				new FieldsFromArray<String>(String.class, 4, 3, 2, 1, 0).extract(testStringArray));
-
-		// check picking fields and reorder
-		String[] crazyOrder = { testStringArray[4], testStringArray[1], testStringArray[2] };
-		arrayEqualityCheck(crazyOrder,
-				new FieldsFromArray<String>(String.class, 4, 1, 2).extract(testStringArray));
-	}
-
-	@Test
-	public void testIntegerArray() {
-		// check single field extraction
-		for (int i = 0; i < testIntegerArray.length; i++) {
-			Integer[] tmp = { testIntegerArray[i] };
-			arrayEqualityCheck(tmp,
-					new FieldsFromArray<Integer>(Integer.class, i).extract(testIntegerArray));
-		}
-
-		// check reverse order
-		Integer[] reverseOrder = new Integer[testIntegerArray.length];
-		for (int i = 0; i < testIntegerArray.length; i++) {
-			reverseOrder[i] = testIntegerArray[testIntegerArray.length - i - 1];
-		}
-		arrayEqualityCheck(reverseOrder,
-				new FieldsFromArray<Integer>(Integer.class, 4, 3, 2, 1, 0)
-						.extract(testIntegerArray));
-
-		// check picking fields and reorder
-		Integer[] crazyOrder = { testIntegerArray[4], testIntegerArray[1], testIntegerArray[2] };
-		arrayEqualityCheck(crazyOrder,
-				new FieldsFromArray<Integer>(Integer.class, 4, 1, 2).extract(testIntegerArray));
-
-	}
-
-	@Test
-	public void testIntArray() {
-		for (int i = 0; i < testIntArray.length; i++) {
-			Integer[] tmp = { testIntArray[i] };
-			arrayEqualityCheck(tmp,
-					new FieldsFromArray<Integer>(Integer.class, i).extract(testIntArray));
-		}
-
-		// check reverse order
-		Integer[] reverseOrder = new Integer[testIntArray.length];
-		for (int i = 0; i < testIntArray.length; i++) {
-			reverseOrder[i] = testIntArray[testIntArray.length - i - 1];
-		}
-		arrayEqualityCheck(reverseOrder,
-				new FieldsFromArray<Integer>(Integer.class, 4, 3, 2, 1, 0).extract(testIntArray));
-
-		// check picking fields and reorder
-		Integer[] crazyOrder = { testIntArray[4], testIntArray[1], testIntArray[2] };
-		arrayEqualityCheck(crazyOrder,
-				new FieldsFromArray<Integer>(Integer.class, 4, 1, 2).extract(testIntArray));
-
-	}
-
-	private void arrayEqualityCheck(Object[] array1, Object[] array2) {
-		assertEquals("The result arrays must have the same length", array1.length, array2.length);
-		for (int i = 0; i < array1.length; i++) {
-			assertEquals("Unequal fields at position " + i, array1[i], array2[i]);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTupleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTupleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTupleTest.java
deleted file mode 100644
index 0379fe0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTupleTest.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.extractor;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple10;
-import org.apache.flink.api.java.tuple.Tuple11;
-import org.apache.flink.api.java.tuple.Tuple12;
-import org.apache.flink.api.java.tuple.Tuple13;
-import org.apache.flink.api.java.tuple.Tuple14;
-import org.apache.flink.api.java.tuple.Tuple15;
-import org.apache.flink.api.java.tuple.Tuple16;
-import org.apache.flink.api.java.tuple.Tuple17;
-import org.apache.flink.api.java.tuple.Tuple18;
-import org.apache.flink.api.java.tuple.Tuple19;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple20;
-import org.apache.flink.api.java.tuple.Tuple21;
-import org.apache.flink.api.java.tuple.Tuple22;
-import org.apache.flink.api.java.tuple.Tuple23;
-import org.apache.flink.api.java.tuple.Tuple24;
-import org.apache.flink.api.java.tuple.Tuple25;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple6;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.tuple.Tuple8;
-import org.apache.flink.api.java.tuple.Tuple9;
-import org.apache.flink.streaming.api.windowing.extractor.FieldsFromTuple;
-import org.junit.Before;
-import org.junit.Test;
-
-public class FieldsFromTupleTest {
-
-	private double[] testDouble;
-
-	@Before
-	public void init() {
-		testDouble = new double[Tuple.MAX_ARITY];
-		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
-			testDouble[i] = i;
-		}
-	}
-
-	@Test
-	public void testUserSpecifiedOrder() throws InstantiationException, IllegalAccessException {
-		Tuple currentTuple = (Tuple) CLASSES[Tuple.MAX_ARITY - 1].newInstance();
-		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
-			currentTuple.setField(testDouble[i], i);
-		}
-
-		double[] expected = { testDouble[5], testDouble[3], testDouble[6], testDouble[7],
-				testDouble[0] };
-		arrayEqualityCheck(expected, new FieldsFromTuple(5, 3, 6, 7, 0).extract(currentTuple));
-
-		double[] expected2 = { testDouble[0], testDouble[Tuple.MAX_ARITY - 1] };
-		arrayEqualityCheck(expected2,
-				new FieldsFromTuple(0, Tuple.MAX_ARITY - 1).extract(currentTuple));
-
-		double[] expected3 = { testDouble[Tuple.MAX_ARITY - 1], testDouble[0] };
-		arrayEqualityCheck(expected3,
-				new FieldsFromTuple(Tuple.MAX_ARITY - 1, 0).extract(currentTuple));
-
-		double[] expected4 = { testDouble[13], testDouble[4], testDouble[5], testDouble[4],
-				testDouble[2], testDouble[8], testDouble[6], testDouble[2], testDouble[8],
-				testDouble[3], testDouble[5], testDouble[2], testDouble[16], testDouble[4],
-				testDouble[3], testDouble[2], testDouble[6], testDouble[4], testDouble[7],
-				testDouble[4], testDouble[2], testDouble[8], testDouble[7], testDouble[2] };
-		arrayEqualityCheck(expected4, new FieldsFromTuple(13, 4, 5, 4, 2, 8, 6, 2, 8, 3, 5, 2, 16,
-				4, 3, 2, 6, 4, 7, 4, 2, 8, 7, 2).extract(currentTuple));
-	}
-
-	private void arrayEqualityCheck(double[] array1, double[] array2) {
-		assertEquals("The result arrays must have the same length", array1.length, array2.length);
-		for (int i = 0; i < array1.length; i++) {
-			assertEquals("Unequal fields at position " + i, array1[i], array2[i], 0d);
-		}
-	}
-
-	private static final Class<?>[] CLASSES = new Class<?>[] { Tuple1.class, Tuple2.class,
-			Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class,
-			Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class,
-			Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class,
-			Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class,
-			Tuple24.class, Tuple25.class };
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicyTest.java
deleted file mode 100644
index 8a7a011..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicyTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class CountEvictionPolicyTest {
-
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	@Test
-	public void testCountEvictionPolicy() {
-		List<Integer> tuples = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
-		int counter;
-
-		// The count policy should not care about the triggered parameter
-		// Therefore its value switches after each use in this test.
-		boolean triggered = false;
-		// the size of the buffer should not matter as well!
-
-		// Test count of different sizes (0..9)
-		for (int i = 0; i < 10; i++) {
-			EvictionPolicy evictionPolicy = new CountEvictionPolicy(i, i);
-			counter = 0;
-
-			// Test first i steps (should not evict)
-			for (int j = 0; j < i; j++) {
-				counter++;
-				assertEquals("Evictionpolicy with count of " + i + " evicted tuples at add nr. "
-						+ counter + ". It should not evict for the first " + i + " adds.", 0,
-						evictionPolicy.notifyEviction(tuples.get(j), (triggered = !triggered),
-								tuples.get(Math.abs((i - j)) % 10)));
-			}
-
-			// Test the next three evictions
-			for (int j = 0; j < 3; j++) {
-				// The first add should evict now
-				counter++;
-				assertEquals("Evictionpolicy with count of " + i
-						+ " did not evict correct number of tuples at the expected pos " + counter
-						+ ".", i, evictionPolicy.notifyEviction(tuples.get(j),
-						(triggered = !triggered), tuples.get(Math.abs((i - j)) % 10)));
-
-				// the next i-1 adds should not evict
-				for (int k = 0; k < i - 1; k++) {
-					counter++;
-					assertEquals("Evictionpolicy with count of " + i
-							+ " evicted tuples at add nr. " + counter, 0,
-							evictionPolicy.notifyEviction(tuples.get(j), (triggered = !triggered),
-									tuples.get(Math.abs((i - j)) % 10)));
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testCountEvictionPolicyStartValuesAndEvictionAmount() {
-
-		// The count policy should not care about the triggered parameter
-		// Therefore its value switches after each use in this test.
-		boolean triggered = false;
-		// the size of the buffer should not matter as well!
-
-		List<Integer> tuples = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
-
-		// Text different eviction amounts (0..3)
-		for (int x = 0; x < 4; x++) {
-
-			// Test count of different sizes (0..9)
-			for (int i = 0; i < 10; i++) {
-
-				int counter = 0;
-
-				// Test different start values (-5..5)
-				for (int j = -5; i < 6; i++) {
-					EvictionPolicy evictionPolicy = new CountEvictionPolicy(i, x, j);
-					// Add tuples without eviction
-					for (int k = 0; k < ((i - j > 0) ? i - j : 0); k++) {
-						counter++;
-						assertEquals("Evictionpolicy with count of " + i
-								+ " did not evict correct number of tuples at the expected pos "
-								+ counter + ".", 0, evictionPolicy.notifyEviction(
-								tuples.get(Math.abs(j)), (triggered = !triggered),
-								tuples.get(Math.abs((i - j)) % 10)));
-					}
-					// Expect eviction
-					counter++;
-					assertEquals("Evictionpolicy with count of " + i
-							+ " did not evict correct number of tuples at the expected pos "
-							+ counter + ".", x, evictionPolicy.notifyEviction(
-							tuples.get(Math.abs(j)), (triggered = !triggered),
-							tuples.get(Math.abs((i - j)) % 10)));
-				}
-			}
-		}
-	}
-
-	@Test
-	public void equalityTest() {
-		assertEquals(new CountEvictionPolicy<Integer>(5, 5, 5), new CountEvictionPolicy<Integer>(5,
-				5, 5));
-
-		assertEquals(new CountEvictionPolicy<Integer>(5, 5), new CountEvictionPolicy<Integer>(5, 5));
-		assertEquals(new CountEvictionPolicy<Integer>(5), new CountEvictionPolicy<Integer>(5));
-
-		assertNotEquals(new CountEvictionPolicy<Integer>(4, 5, 5),
-				new CountEvictionPolicy<Integer>(5, 5, 5));
-		assertNotEquals(new CountEvictionPolicy<Integer>(5, 5, 5),
-				new CountEvictionPolicy<Integer>(5, 4, 5));
-
-		assertNotEquals(new CountEvictionPolicy<Integer>(5, 5, 5),
-				new CountEvictionPolicy<Integer>(5, 5, 4));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicyTest.java
deleted file mode 100644
index ce5ae3b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicyTest.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.streaming.api.windowing.helper.Count;
-import org.junit.Test;
-
-public class CountTriggerPolicyTest {
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testCountTriggerPolicy() {
-
-		List tuples = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
-		int counter;
-
-		// Test count of different sizes (0..9)
-		for (int i = 0; i < 10; i++) {
-			TriggerPolicy triggerPolicy = Count.of(i).toTrigger();
-			counter = 0;
-
-			// Test first i steps (should not trigger)
-			for (int j = 0; j < i; j++) {
-				counter++;
-				assertFalse("Triggerpolicy with count of " + i + " triggered at add nr. " + counter
-						+ ". It should not trigger for the first " + i + " adds.",
-						triggerPolicy.notifyTrigger(tuples.get(j)));
-			}
-
-			// Test the next three triggers
-			for (int j = 0; j < 3; j++) {
-				// The first add should trigger now
-				counter++;
-				assertTrue("Triggerpolicy with count of " + i
-						+ " did not trigger at the expected pos " + counter + ".",
-						triggerPolicy.notifyTrigger(tuples.get(j)));
-
-				// the next i-1 adds should not trigger
-				for (int k = 0; k < i - 1; k++) {
-					counter++;
-					assertFalse("Triggerpolicy with count of " + i + " triggered at add nr. "
-							+ counter, triggerPolicy.notifyTrigger(tuples.get(k)));
-				}
-			}
-		}
-	}
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testCountTriggerPolicyStartValues() {
-
-		List tuples = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
-
-		// Test count of different sizes (0..9)
-		for (int i = 0; i < 10; i++) {
-
-			// Test different start values (-5..5)
-			for (int j = -5; i < 6; i++) {
-				TriggerPolicy triggerPolicy = new CountTriggerPolicy(i, j);
-				// Add tuples without trigger
-				for (int k = 0; k < ((i - j > 0) ? i - j : 0); k++) {
-					assertFalse("Triggerpolicy with count of " + i + " and start value of " + j
-							+ " triggered at add nr. " + (k + 1),
-							triggerPolicy.notifyTrigger(tuples.get(k % 10)));
-				}
-				// Expect trigger
-				assertTrue("Triggerpolicy with count of " + i + "and start value of " + j
-						+ " did not trigger at the expected position.",
-						triggerPolicy.notifyTrigger(tuples.get(0)));
-			}
-		}
-	}
-
-	@Test
-	public void equalityTest() {
-		assertEquals(new CountTriggerPolicy<Integer>(5, 5), new CountTriggerPolicy<Integer>(5, 5));
-
-		assertEquals(new CountTriggerPolicy<Integer>(5, 5), new CountTriggerPolicy<Integer>(5, 5));
-		assertEquals(new CountTriggerPolicy<Integer>(5), new CountTriggerPolicy<Integer>(5));
-
-		assertNotEquals(new CountTriggerPolicy<Integer>(4, 5),
-				new CountTriggerPolicy<Integer>(5, 5));
-		assertNotEquals(new CountTriggerPolicy<Integer>(5, 5),
-				new CountTriggerPolicy<Integer>(5, 4));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicyTest.java
deleted file mode 100644
index 9ec4644..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicyTest.java
+++ /dev/null
@@ -1,88 +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.flink.streaming.api.windowing.policy;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.junit.Test;
-
-import java.util.List;
-import java.util.Arrays;
-
-import static org.junit.Assert.*;
-
-public class DeltaPolicyTest {
-
-	//Dummy serializer, this is not used because the tests are done locally
-	private final static TypeSerializer<Tuple2<Integer, Integer>> SERIALIZER = null;
-
-	@SuppressWarnings({ "serial", "unchecked", "rawtypes" })
-	@Test
-	public void testDelta() {
-		DeltaPolicy deltaPolicy = new DeltaPolicy(new DeltaFunction<Tuple2<Integer, Integer>>() {
-			@Override
-			public double getDelta(Tuple2<Integer, Integer> oldDataPoint,
-					Tuple2<Integer, Integer> newDataPoint) {
-				return (double) newDataPoint.f0 - oldDataPoint.f0;
-			}
-		}, new Tuple2(0, 0), 2, SERIALIZER);
-
-		List<Tuple2> tuples = Arrays.asList(new Tuple2(1, 0), new Tuple2(2, 0), new Tuple2(3, 0),
-				new Tuple2(6, 0));
-
-		assertFalse(deltaPolicy.notifyTrigger(tuples.get(0)));
-		assertEquals(0, deltaPolicy.notifyEviction(tuples.get(0), false, 0));
-
-		assertFalse(deltaPolicy.notifyTrigger(tuples.get(1)));
-		assertEquals(0, deltaPolicy.notifyEviction(tuples.get(1), false, 1));
-
-		assertTrue(deltaPolicy.notifyTrigger(tuples.get(2)));
-		assertEquals(1, deltaPolicy.notifyEviction(tuples.get(2), true, 2));
-
-		assertTrue(deltaPolicy.notifyTrigger(tuples.get(3)));
-		assertEquals(2, deltaPolicy.notifyEviction(tuples.get(3), true, 2));
-	}
-
-	@Test
-	public void testEquality() {
-
-		DeltaFunction<Tuple2<Integer, Integer>> df = new DeltaFunction<Tuple2<Integer, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public double getDelta(Tuple2<Integer, Integer> oldDataPoint,
-					Tuple2<Integer, Integer> newDataPoint) {
-				return (double) newDataPoint.f0 - oldDataPoint.f0;
-			}
-		};
-
-		assertEquals(new DeltaPolicy<Tuple2<Integer, Integer>>(df, new Tuple2<Integer, Integer>(0,
-				0), 2, SERIALIZER), new DeltaPolicy<Tuple2<Integer, Integer>>(df, new Tuple2<Integer, Integer>(
-				0, 0), 2, SERIALIZER));
-
-		assertNotEquals(new DeltaPolicy<Tuple2<Integer, Integer>>(df, new Tuple2<Integer, Integer>(
-				0, 1), 2, SERIALIZER), new DeltaPolicy<Tuple2<Integer, Integer>>(df,
-				new Tuple2<Integer, Integer>(0, 0), 2, SERIALIZER));
-
-		assertNotEquals(new DeltaPolicy<Tuple2<Integer, Integer>>(df, new Tuple2<Integer, Integer>(0,
-				0), 2, SERIALIZER), new DeltaPolicy<Tuple2<Integer, Integer>>(df, new Tuple2<Integer, Integer>(
-				0, 0), 3, SERIALIZER));
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicyTest.java
deleted file mode 100644
index 3214aa7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicyTest.java
+++ /dev/null
@@ -1,186 +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.flink.streaming.api.windowing.policy;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.junit.Test;
-
-public class MultiEvictionPolicyTest {
-
-	private final List<Integer> tuples = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
-
-	private final CloneableEvictionPolicy<Integer> evictionPolicy1 = new CountEvictionPolicy<Integer>(
-			7, 3);
-	private final CloneableEvictionPolicy<Integer> evictionPolicy2 = new CountEvictionPolicy<Integer>(
-			3, 1);
-	private final CloneableEvictionPolicy<Integer> evictionPolicy3 = new CountEvictionPolicy<Integer>(
-			5, 2);
-
-	private final CloneableEvictionPolicy<Integer> activeEvictionPolicy1 = new ActiveCloneableEvictionPolicyWrapper<Integer>(
-			evictionPolicy1);
-	private final CloneableEvictionPolicy<Integer> activeEvictionPolicy2 = new ActiveCloneableEvictionPolicyWrapper<Integer>(
-			evictionPolicy2);
-	private final CloneableEvictionPolicy<Integer> activeEvictionPolicy3 = new ActiveCloneableEvictionPolicyWrapper<Integer>(
-			evictionPolicy3);
-
-	// From policies specified above the expected output is:
-	// 1.: 0000000300
-	// 2.: 0001111111
-	// 3.: 0000020202
-	private final Integer[] maxResult = { 0, 0, 0, 1, 1, 2, 1, 3, 1, 2 };
-	private final Integer[] minResult = { 0, 0, 0, 0, 0, 0, 0, 1, 0, 0 };
-	private final Integer[] sumResult = { 0, 0, 0, 1, 1, 3, 1, 6, 1, 3 };
-	private final Integer[] priorityResult = { 0, 0, 0, 1, 1, 1, 1, 3, 1, 1 };
-
-	/*
-	 * Test cases for not active policies
-	 */
-
-	@Test
-	public void notActiveEvictionMAXStrategyTest() {
-		runNotActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.MAX, maxResult);
-	}
-
-	@Test
-	public void notActiveEvictionMINStrategyTest() {
-		runNotActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.MIN, minResult);
-	}
-
-	@Test
-	public void notActiveEvictionSUMStrategyTest() {
-		runNotActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.SUM, sumResult);
-	}
-
-	@Test
-	public void notActiveEvictionPRIORITYStrategyTest() {
-		runNotActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.PRIORITY, priorityResult);
-	}
-
-	/*
-	 * Test cases for active policies
-	 */
-
-	@Test
-	public void activeEvictionMAXStrategyTest() {
-		runActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.MAX, maxResult);
-	}
-
-	@Test
-	public void activeEvictionMINStrategyTest() {
-		runActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.MIN, minResult);
-	}
-
-	@Test
-	public void activeEvictionSUMStrategyTest() {
-		runActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.SUM, sumResult);
-	}
-
-	@Test
-	public void activeEvictionPRIORITYStrategyTest() {
-		runActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy.PRIORITY, priorityResult);
-	}
-
-	/**
-	 * Helper method: It runs the test with the given input using the not active
-	 * policies and applies the strategy defined in the parameter.
-	 * 
-	 * @param strategy
-	 *            the eviction strategy to be used
-	 * @param expectedResult
-	 *            the result we expect
-	 */
-	private void runNotActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy strategy,
-			Integer[] expectedResult) {
-		@SuppressWarnings("unchecked")
-		MultiEvictionPolicy<Integer> multiEviction = new MultiEvictionPolicy<Integer>(strategy,
-				evictionPolicy1.clone(), evictionPolicy2.clone(), evictionPolicy3.clone());
-
-		List<Integer> result = new LinkedList<Integer>();
-
-		int buffersize = 0;
-		for (Integer tuple : tuples) {
-			// The buffer size should not matter, but we keep it for the case of
-			// later policy changes.
-			// The trigger does not matter. Always set it to false.
-			int eviction = multiEviction.notifyEviction(tuple, false, buffersize);
-			buffersize -= eviction;
-			result.add(eviction);
-
-			if (buffersize < 0) {
-				buffersize = 0;
-			}
-
-			buffersize++;
-		}
-
-		arrayEqualityCheck(expectedResult, result.toArray());
-	}
-
-	/**
-	 * Helper method: It runs the test with the given input using the active
-	 * policies and applies the strategy defined in the parameter.
-	 * 
-	 * @param strategy
-	 *            the eviction strategy to be used
-	 * @param expectedResult
-	 *            the result we expect
-	 */
-	private void runActiveEvictionTest(MultiEvictionPolicy.EvictionStrategy strategy,
-			Integer[] expectedResult) {
-		@SuppressWarnings("unchecked")
-		MultiEvictionPolicy<Integer> multiEviction = new MultiEvictionPolicy<Integer>(strategy,
-				activeEvictionPolicy1.clone(), activeEvictionPolicy2.clone(),
-				activeEvictionPolicy3.clone());
-
-		List<Integer> result = new LinkedList<Integer>();
-
-		int buffersize = 0;
-		for (Integer tuple : tuples) {
-			// The buffer size should not matter, but we keep it for the case of
-			// later policy changes.
-			// The trigger does not matter. Always set it to false.
-			int eviction = multiEviction.notifyEvictionWithFakeElement(tuple, buffersize);
-			buffersize -= eviction;
-			result.add(eviction);
-
-			if (buffersize < 0) {
-				buffersize = 0;
-			}
-
-			buffersize++;
-		}
-
-		arrayEqualityCheck(expectedResult, result.toArray());
-	}
-
-	private void arrayEqualityCheck(Object[] array1, Object[] array2) {
-		assertEquals(
-				"The result arrays must have the same length. (Expected: " + Arrays.asList(array1)
-						+ "; Actual: " + Arrays.asList(array2) + ")", array1.length, array2.length);
-		for (int i = 0; i < array1.length; i++) {
-			assertEquals("Unequal fields at position " + i + "(Expected: " + Arrays.asList(array1)
-					+ "; Actual: " + Arrays.asList(array2) + ")", array1[i], array2[i]);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicyTest.java
deleted file mode 100644
index 4448b59..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicyTest.java
+++ /dev/null
@@ -1,245 +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.flink.streaming.api.windowing.policy;
-
-import com.google.common.collect.Sets;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class MultiTriggerPolicyTest {
-
-	/**
-	 * This constant defines the timeout for the test of the start ups of the
-	 * active trigger policy Threads.
-	 */
-	private static final int TIMEOUT = 120000;
-
-	// Use this to increase the timeout to be as long as possible.
-	// private static final int TIMEOUT=Integer.MAX_VALUE;
-
-	/**
-	 * This test covers all regular notify call. It takes no fake elements into
-	 * account.
-	 */
-	@Test
-	public void testWithoutActivePolicies() {
-		List<Integer> tuples = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15,
-				16);
-
-		TriggerPolicy<Integer> firstPolicy = new CountTriggerPolicy<Integer>(3);
-		TriggerPolicy<Integer> secondPolicy = new CountTriggerPolicy<Integer>(5);
-		TriggerPolicy<Integer> thirdPolicy = new CountTriggerPolicy<Integer>(8);
-		@SuppressWarnings("unchecked")
-		TriggerPolicy<Integer> multiTrigger = new MultiTriggerPolicy<Integer>(firstPolicy,
-				secondPolicy, thirdPolicy);
-
-		// From above policies the expected output is (first element is 0):
-		// first: 3, 6, 9, 12, 15,...
-		// second: 5, 10, 15,...
-		// third: 8, 16, 24,...
-		// combination: 3,5,6,8,9,10,12,15,16
-		List<Integer> expectedResult = Arrays.asList(3, 5, 6, 8, 9, 10, 12, 15, 16);
-		List<Integer> actualResult = new LinkedList<Integer>();
-
-		for (int i = 0; i < tuples.size(); i++) {
-			if (multiTrigger.notifyTrigger(tuples.get(i))) {
-				actualResult.add(i);
-			}
-		}
-
-		// check equal sizes
-		assertTrue("The expected result list and the actual result list must have the same size,"
-				+ " but they are different. (expected: " + expectedResult.size() + "; actual: "
-				+ actualResult.size() + "). Actual result is: " + actualResult
-				+ " Expected result is: " + expectedResult,
-				expectedResult.size() == actualResult.size());
-
-		// check equal elements within result list/expected list
-		for (int i = 0; i < expectedResult.size(); i++) {
-			assertTrue("The actual and the expected result does not match at position " + i
-					+ ". (expected: " + expectedResult.get(i) + "; actual: " + actualResult.get(i)
-					+ "). Actual result is: " + actualResult + " Expected result is: "
-					+ expectedResult, expectedResult.get(i) == actualResult.get(i));
-		}
-	}
-
-	/**
-	 * This test covers the pre-notify calls to active policies. I takes no
-	 * regular notify into account.
-	 */
-	@Test
-	public void testWithActivePolicies() {
-
-		// create some test data
-		Integer[] times = { 1, 3, 20, 26 };
-
-		// create a timestamp
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		// create policy
-		TimeTriggerPolicy<Integer> firstPolicy = new TimeTriggerPolicy<Integer>(5,
-				new TimestampWrapper<Integer>(timeStamp, 0));
-		TimeTriggerPolicy<Integer> secondPolicy = new TimeTriggerPolicy<Integer>(10,
-				new TimestampWrapper<Integer>(timeStamp, 0));
-		TimeTriggerPolicy<Integer> thirdPolicy = new TimeTriggerPolicy<Integer>(22,
-				new TimestampWrapper<Integer>(timeStamp, 0));
-		@SuppressWarnings("unchecked")
-		MultiTriggerPolicy<Integer> multiTrigger = new MultiTriggerPolicy<Integer>(firstPolicy,
-				secondPolicy, thirdPolicy);
-
-		// expected result
-		// Long[][] result1 = { {}, {}, { 4L, 9L, 14L, 19L }, { 24L } };
-		// Long[][] result2 = { {}, {}, { 9L, 19L }, { } };
-		// Long[][] result3 = { {}, {}, { }, { 21L } };
-		Long[][] result = { {}, {}, { 4L, 9L, 14L, 19L, 9L, 19L }, { 24L, 21L } };
-
-		// call policy
-		for (int i = 0; i < times.length; i++) {
-			arrayEqualityCheck(result[i], multiTrigger.preNotifyTrigger(times[i]));
-			multiTrigger.notifyTrigger(times[i]);
-		}
-	}
-
-	/**
-	 * This test verifies, that nestet active trigger runnables are started
-	 * correctly.
-	 */
-	@Test
-	public void testActiveTriggerRunnables() throws InterruptedException {
-		TriggerPolicy<Integer> firstPolicy = new ActiveTriggerWithRunnable(1);
-		TriggerPolicy<Integer> secondPolicy = new ActiveTriggerWithRunnable(2);
-		TriggerPolicy<Integer> thirdPolicy = new ActiveTriggerWithRunnable(3);
-		@SuppressWarnings("unchecked")
-		ActiveTriggerPolicy<Integer> multiTrigger = new MultiTriggerPolicy<Integer>(firstPolicy,
-				secondPolicy, thirdPolicy);
-
-		MyCallbackClass cb = new MyCallbackClass(3);
-		Runnable runnable = multiTrigger.createActiveTriggerRunnable(cb);
-		new Thread(runnable).start();
-
-		assertTrue("Even after " + TIMEOUT + "ms not all active policy runnables were started.",
-				cb.check(TIMEOUT, 1, 2, 3));
-	}
-
-	private void arrayEqualityCheck(Object[] array1, Object[] array2) {
-		assertEquals(
-				"The result arrays must have the same length. (Expected: " + Arrays.asList(array1)
-						+ "; Actual: " + Arrays.asList(array2) + ")", array1.length, array2.length);
-		for (int i = 0; i < array1.length; i++) {
-			assertEquals("Unequal fields at position " + i + "(Expected: " + Arrays.asList(array1)
-					+ "; Actual: " + Arrays.asList(array2) + ")", array1[i], array2[i]);
-		}
-	}
-
-	/**
-	 * This helper class is used to simulate active triggers which produce own
-	 * runnables.
-	 */
-	@SuppressWarnings("serial")
-	private class ActiveTriggerWithRunnable implements ActiveTriggerPolicy<Integer> {
-
-		private final int id;
-
-		public ActiveTriggerWithRunnable(int id) {
-			this.id = id;
-		}
-
-		@Override
-		public boolean notifyTrigger(Integer datapoint) {
-			// This method is not uses for any test case
-			return false;
-		}
-
-		@Override
-		public Object[] preNotifyTrigger(Integer datapoint) {
-			// This method is not used for any test case
-			return null;
-		}
-
-		@Override
-		public Runnable createActiveTriggerRunnable(final ActiveTriggerCallback callback) {
-			return new Runnable() {
-				@Override
-				public void run() {
-					callback.sendFakeElement(id);
-				}
-			};
-		}
-	}
-
-	/**
-	 * This callback class is used to checked whether all nested policy runnable
-	 * started up.
-	 */
-	private class MyCallbackClass implements ActiveTriggerCallback {
-
-		private final Set<Integer> received = Sets
-				.newSetFromMap(new ConcurrentHashMap<Integer, Boolean>());
-
-		private final CountDownLatch sync;
-
-		public MyCallbackClass(int numberOfExpectedElements) {
-			checkArgument(numberOfExpectedElements >= 0);
-			this.sync = new CountDownLatch(numberOfExpectedElements);
-		}
-
-		@Override
-		public void sendFakeElement(Object datapoint) {
-			received.add((Integer) datapoint);
-
-			sync.countDown();
-		}
-
-		public boolean check(int timeout, int... expectedIds) throws InterruptedException {
-			// Wait for all elements
-			sync.await(timeout, TimeUnit.MILLISECONDS);
-
-			// Check received all expected ids
-			assertEquals(expectedIds.length, received.size());
-
-			for (int id : expectedIds) {
-				if (!received.contains(id)) {
-					return false;
-				}
-			}
-
-			return true;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicyTest.java
deleted file mode 100644
index fda9cd3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicyTest.java
+++ /dev/null
@@ -1,155 +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.flink.streaming.api.windowing.policy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.windowing.extractor.Extractor;
-import org.apache.flink.streaming.api.windowing.extractor.FieldFromTuple;
-import org.junit.Test;
-
-public class PunctuationPolicyTest {
-
-	// This value should not effect the policy. It is changed at each call to
-	// verify this.
-	private boolean triggered = false;
-
-	@Test
-	public void PunctuationTriggerTestWithoutExtraction() {
-		PunctuationPolicy<Object, Object> policy = new PunctuationPolicy<Object, Object>(
-				new TestObject(0));
-		assertTrue("The present punctuation was not detected. (POS 1)",
-				policy.notifyTrigger(new TestObject(0)));
-		assertFalse("There was a punctuation detected which wasn't present. (POS 2)",
-				policy.notifyTrigger(new TestObject(1)));
-		policy.toString();
-	}
-
-	@Test
-	public void PunctuationTriggerTestWithExtraction() {
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		PunctuationPolicy<Tuple2<Object, Object>, Object> policy = new PunctuationPolicy<Tuple2<Object, Object>, Object>(
-				new TestObject(0), new FieldFromTuple(0));
-		assertTrue("The present punctuation was not detected. (POS 3)",
-				policy.notifyTrigger(new Tuple2<Object, Object>(new TestObject(0),
-						new TestObject(1))));
-		assertFalse("There was a punctuation detected which wasn't present. (POS 4)",
-				policy.notifyTrigger(new Tuple2<Object, Object>(new TestObject(1),
-						new TestObject(0))));
-	}
-
-	@Test
-	public void PunctuationEvictionTestWithoutExtraction() {
-		// The current buffer size should not effect the test. It's therefore
-		// always 0 here.
-
-		PunctuationPolicy<Object, Object> policy = new PunctuationPolicy<Object, Object>(
-				new TestObject(0));
-		assertEquals(
-				"The present punctuation was not detected or the number of deleted tuples was wrong. (POS 5)",
-				0, policy.notifyEviction(new TestObject(0), (triggered = !triggered), 0));
-		for (int i = 0; i < 10; i++) {
-			for (int j = 0; j < i; j++) {
-				assertEquals("There was a punctuation detected which wasn't present. (POS 6)", 0,
-						policy.notifyEviction(new TestObject(1), (triggered = !triggered), 0));
-			}
-			assertEquals(
-					"The present punctuation was not detected or the number of deleted tuples was wrong. (POS 7)",
-					i + 1, policy.notifyEviction(new TestObject(0), (triggered = !triggered), 0));
-		}
-	}
-
-	@Test
-	public void PunctuationEvictionTestWithExtraction() {
-		// The current buffer size should not effect the test. It's therefore
-		// always 0 here.
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		PunctuationPolicy<Tuple2<Object, Object>, Object> policy = new PunctuationPolicy<Tuple2<Object, Object>, Object>(
-				new TestObject(0), new FieldFromTuple(0));
-		assertEquals(
-				"The present punctuation was not detected or the number of deleted tuples was wrong. (POS 10)",
-				0, policy.notifyEviction(new Tuple2<Object, Object>(new TestObject(0),
-						new TestObject(1)), (triggered = !triggered), 0));
-		for (int i = 0; i < 10; i++) {
-			for (int j = 0; j < i; j++) {
-				assertEquals("There was a punctuation detected which wasn't present. (POS 9)", 0,
-						policy.notifyEviction(new Tuple2<Object, Object>(new TestObject(1),
-								new TestObject(0)), (triggered = !triggered), 0));
-			}
-			assertEquals(
-					"The present punctuation was not detected or the number of deleted tuples was wrong. (POS 10)",
-					i + 1, policy.notifyEviction(new Tuple2<Object, Object>(new TestObject(0),
-							new TestObject(1)), (triggered = !triggered), 0));
-		}
-	}
-
-	@Test
-	public void testEquals() {
-		Extractor<Integer, Integer> extractor = new Extractor<Integer, Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Integer extract(Integer in) {
-				return in;
-			}
-		};
-
-		assertEquals(new PunctuationPolicy<Integer, Integer>(4),
-				new PunctuationPolicy<Integer, Integer>(4));
-		assertNotEquals(new PunctuationPolicy<Integer, Integer>(4),
-				new PunctuationPolicy<Integer, Integer>(5));
-
-		assertNotEquals(new PunctuationPolicy<Integer, Integer>(4, extractor),
-				new PunctuationPolicy<Integer, Integer>(4));
-
-		assertEquals(new PunctuationPolicy<Integer, Integer>(4, extractor),
-				new PunctuationPolicy<Integer, Integer>(4, extractor));
-
-		assertNotEquals(new PunctuationPolicy<Integer, Integer>(4),
-				new PunctuationPolicy<Integer, Integer>(4, extractor));
-
-	}
-
-	private class TestObject {
-
-		private int id;
-
-		public TestObject(int id) {
-			this.id = id;
-		}
-
-		@Override
-		public boolean equals(Object o) {
-			if (o instanceof TestObject && ((TestObject) o).getId() == this.id) {
-				return true;
-			} else {
-				return false;
-			}
-		}
-
-		public int getId() {
-			return id;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicyTest.java
deleted file mode 100644
index 08a5c32..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicyTest.java
+++ /dev/null
@@ -1,164 +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.flink.streaming.api.windowing.policy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.LinkedList;
-
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.junit.Test;
-
-public class TimeEvictionPolicyTest {
-
-	@Test
-	public void timeEvictionTest() {
-		// create some test data
-		Integer[] times = { 1, 3, 4, 6, 7, 9, 14, 20, 21, 22, 30, 31, 33, 36, 40, 41, 42, 43, 44,
-				45, 47, 55 };
-		Integer[] numToDelete = { 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 2, 0, 0, 0, 0, 3 };
-
-		// create a timestamp
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		// test different granularity
-		for (long granularity = 0; granularity < 40; granularity++) {
-			// create policy
-			TimeEvictionPolicy<Integer> policy = new TimeEvictionPolicy<Integer>(granularity,
-					new TimestampWrapper<Integer>(timeStamp, 0));
-
-			// The trigger status should not effect the policy. Therefore, it's
-			// value is changed after each usage.
-			boolean triggered = false;
-
-			// The eviction should work similar with both, fake and real
-			// elements. Which kind is used is changed on every 3rd element in
-			// this test.
-			int fakeAndRealCounter = 0;
-			boolean fake = false;
-
-			// test by adding values
-			LinkedList<Integer> buffer = new LinkedList<Integer>();
-			for (int i = 0; i < times.length; i++) {
-
-				// check if the current element should be a fake
-				fakeAndRealCounter++;
-				if (fakeAndRealCounter > 2) {
-					fake = !fake;
-					fakeAndRealCounter = 0;
-				}
-
-				int result;
-
-				if (fake) {
-					// Notify eviction with fake element
-					result = policy.notifyEvictionWithFakeElement(times[i], buffer.size());
-				} else {
-					// Notify eviction with real element
-					result = policy.notifyEviction(times[i], (triggered = !triggered),
-							buffer.size());
-				}
-
-				// handle correctness of eviction
-				for (; result > 0 && !buffer.isEmpty(); result--) {
-					if (buffer.getFirst() <= times[i] - granularity) {
-						buffer.removeFirst();
-					} else {
-						fail("The policy wanted to evict time " + buffer.getFirst()
-								+ " while the current time was " + times[i]
-								+ "and the granularity was " + granularity);
-					}
-				}
-
-				// test that all required evictions have been done
-				if (!buffer.isEmpty()) {
-					assertTrue("The policy did not evict " + buffer.getFirst()
-							+ " while the current time was " + times[i]
-							+ " and the granularity was " + granularity,
-							(buffer.getFirst() >= times[i] - granularity));
-				}
-
-				// test influence of other evictions
-				for (int j = numToDelete[i % numToDelete.length]; j > 0; j--) {
-					if (!buffer.isEmpty()) {
-						buffer.removeFirst();
-					}
-				}
-
-				// add current element to buffer if it is no fake
-				if (!fake) {
-					buffer.add(times[i]);
-				}
-
-			}
-		}
-	}
-
-	@Test
-	public void equalsTest() {
-
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp2 = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		assertEquals(
-				new TimeEvictionPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp, 0)),
-				new TimeEvictionPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp, 0)));
-
-		assertNotEquals(new TimeEvictionPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp,
-				0)), new TimeEvictionPolicy<Integer>(5,
-				new TimestampWrapper<Integer>(timeStamp2, 0)));
-
-		assertNotEquals(new TimeEvictionPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp,
-				0)),
-				new TimeEvictionPolicy<Integer>(2, new TimestampWrapper<Integer>(timeStamp, 0)));
-
-		assertNotEquals(new TimeEvictionPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp,
-				0)),
-				new TimeEvictionPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp, 3)));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicyTest.java
deleted file mode 100644
index 5b26854..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicyTest.java
+++ /dev/null
@@ -1,156 +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.flink.streaming.api.windowing.policy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.junit.Test;
-
-public class TimeTriggerPolicyTest {
-
-	@Test
-	public void timeTriggerRegularNotifyTest() {
-		// create some test data
-		Integer[] times = { 1, 3, 4, 6, 7, 9, 14, 20, 21, 22, 30 };
-
-		// create a timestamp
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		// test different granularity
-		for (long granularity = 0; granularity < 31; granularity++) {
-			// create policy
-
-			TriggerPolicy<Integer> policy = new TimeTriggerPolicy<Integer>(granularity,
-					new TimestampWrapper<Integer>(timeStamp, 0));
-
-			// remember window border
-			long currentTime = 0;
-
-			// test by adding values
-			for (int i = 0; i < times.length; i++) {
-				boolean result = policy.notifyTrigger(times[i]);
-				// start time is included, but end time is excluded: >=
-				if (times[i] >= currentTime + granularity) {
-					if (granularity != 0) {
-						currentTime = times[i] - ((times[i] - currentTime) % granularity);
-					}
-					assertTrue("The policy did not trigger at pos " + i + " (current time border: "
-							+ currentTime + "; current granularity: " + granularity
-							+ "; data point time: " + times[i] + ")", result);
-				} else {
-					assertFalse("The policy triggered wrong at pos " + i
-							+ " (current time border: " + currentTime + "; current granularity: "
-							+ granularity + "; data point time: " + times[i] + ")", result);
-				}
-			}
-		}
-
-	}
-
-	@Test
-	public void equalsTest() {
-
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp2 = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		assertEquals(
-				new TimeTriggerPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp, 0)),
-				new TimeTriggerPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp, 0)));
-
-		assertNotEquals(new TimeTriggerPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp,
-				0)),
-				new TimeTriggerPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp2, 0)));
-
-		assertNotEquals(new TimeTriggerPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp,
-				0)), new TimeTriggerPolicy<Integer>(2, new TimestampWrapper<Integer>(timeStamp, 0)));
-
-		assertNotEquals(new TimeTriggerPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp,
-				0)), new TimeTriggerPolicy<Integer>(5, new TimestampWrapper<Integer>(timeStamp, 3)));
-
-		assertEquals(SystemTimestamp.getWrapper(), SystemTimestamp.getWrapper());
-	}
-
-	@Test
-	public void timeTriggerPreNotifyTest() {
-		// create some test data
-		Integer[] times = { 1, 3, 20, 26 };
-
-		// create a timestamp
-		@SuppressWarnings("serial")
-		Timestamp<Integer> timeStamp = new Timestamp<Integer>() {
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-
-		};
-
-		// create policy
-		TimeTriggerPolicy<Integer> policy = new TimeTriggerPolicy<Integer>(5,
-				new TimestampWrapper<Integer>(timeStamp, 0));
-
-		// expected result
-		Long[][] result = { {}, {}, { 4L, 9L, 14L, 19L }, { 24L } };
-
-		// call policy
-		for (int i = 0; i < times.length; i++) {
-			arrayEqualityCheck(result[i], policy.preNotifyTrigger(times[i]));
-			policy.notifyTrigger(times[i]);
-		}
-	}
-
-	private void arrayEqualityCheck(Object[] array1, Object[] array2) {
-		assertEquals("The result arrays must have the same length", array1.length, array2.length);
-		for (int i = 0; i < array1.length; i++) {
-			assertEquals("Unequal fields at position " + i, array1[i], array2[i]);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicyTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicyTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicyTest.java
deleted file mode 100644
index a3a7d73..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicyTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-
-public class TumblingEvictionPolicyTest {
-
-	@Test
-	public void testTumblingEviction() {
-		EvictionPolicy<Integer> policy = new TumblingEvictionPolicy<Integer>();
-
-		int counter = 0;
-
-		for (int i = 0; i < 10; i++) {
-			for (int j = 0; j < i; j++) {
-				assertEquals(0, policy.notifyEviction(0, false, counter++));
-			}
-			assertEquals(counter, policy.notifyEviction(0, true, counter));
-			counter = 1;
-		}
-
-		assertEquals(new TumblingEvictionPolicy<Integer>(), new TumblingEvictionPolicy<Integer>());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java
deleted file mode 100644
index 6bc0e30..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBufferTest.java
+++ /dev/null
@@ -1,86 +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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Test;
-
-public class BasicWindowBufferTest {
-
-	@Test
-	public void testEmitWindow() throws Exception {
-
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-		List<StreamWindow<Integer>> collected = collector.getCollected();
-
-		WindowBuffer<Integer> wb = new BasicWindowBuffer<Integer>();
-
-		wb.store(2);
-		wb.store(10);
-
-		wb.emitWindow(collector);
-
-		assertEquals(1, collected.size());
-		assertEquals(StreamWindow.fromElements(2, 10), collected.get(0));
-
-		wb.store(4);
-		wb.evict(2);
-
-		wb.emitWindow(collector);
-
-		assertEquals(2, collected.size());
-		assertEquals(StreamWindow.fromElements(4), collected.get(1));
-
-		wb.evict(1);
-
-		wb.emitWindow(collector);
-		assertEquals(2, collected.size());
-	}
-
-	public static class TestOutput<T> implements Output<StreamRecord<T>> {
-
-		private final List<T> collected = new ArrayList<T>();
-
-		@Override
-		public void collect(StreamRecord<T> record) {
-			collected.add(record.getValue());
-		}
-
-		@Override
-		public void close() {
-		}
-
-		public List<T> getCollected() {
-			return collected;
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java
deleted file mode 100644
index 8430499..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducerTest.java
+++ /dev/null
@@ -1,157 +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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-
-import org.junit.Test;
-
-public class JumpingCountGroupedPreReducerTest {
-
-	TypeInformation<Tuple2<Integer, Integer>> type = TypeExtractor
-			.getForObject(new Tuple2<Integer, Integer>(1, 1));
-	TypeSerializer<Tuple2<Integer, Integer>> serializer = type.createSerializer(null);
-
-	KeySelector<Tuple2<Integer, Integer>, ?> key = KeySelectorUtil.getSelectorForKeys(
-			new Keys.ExpressionKeys<Tuple2<Integer, Integer>>(new int[] { 0 }, type), type, null);
-
-	Reducer reducer = new Reducer();
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testEmitWindow() throws Exception {
-
-		List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
-		inputs.add(new Tuple2<Integer, Integer>(1, 1));
-		inputs.add(new Tuple2<Integer, Integer>(0, 0));
-		inputs.add(new Tuple2<Integer, Integer>(1, -1));
-		inputs.add(new Tuple2<Integer, Integer>(1, -2));
-		inputs.add(new Tuple2<Integer, Integer>(100, -200));
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-		List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();
-
-		WindowBuffer<Tuple2<Integer, Integer>> wb = new JumpingCountGroupedPreReducer<Tuple2<Integer, Integer>>(
-				reducer, key, serializer, 1);
-
-		wb.store(serializer.copy(inputs.get(4)));
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.emitWindow(collector);
-
-		assertEquals(1, collected.size());
-
-		assertSetEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(1, 1),
-				new Tuple2<Integer, Integer>(0, 0)), collected.get(0));
-
-		wb.store(serializer.copy(inputs.get(4)));
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.store(serializer.copy(inputs.get(2)));
-
-		// Nothing should happen here
-		wb.evict(3);
-
-		wb.store(serializer.copy(inputs.get(3)));
-
-		wb.emitWindow(collector);
-
-		assertEquals(2, collected.size());
-
-		assertSetEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(3, -2),
-				new Tuple2<Integer, Integer>(0, 0)), collected.get(1));
-
-		// Test whether function is mutating inputs or not
-		assertEquals(2, reducer.allInputs.size());
-		assertEquals(reducer.allInputs.get(0), inputs.get(2));
-		assertEquals(reducer.allInputs.get(1), inputs.get(3));
-
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testEmitWindow2() throws Exception {
-
-		List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
-		inputs.add(new Tuple2<Integer, Integer>(1, 1));
-		inputs.add(new Tuple2<Integer, Integer>(0, 0));
-		inputs.add(new Tuple2<Integer, Integer>(1, -1));
-		inputs.add(new Tuple2<Integer, Integer>(1, -2));
-		inputs.add(new Tuple2<Integer, Integer>(100, -200));
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-		List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();
-
-		WindowBuffer<Tuple2<Integer, Integer>> wb = new JumpingCountGroupedPreReducer<Tuple2<Integer, Integer>>(
-				reducer, key, serializer, 1).sequentialID();
-
-		wb.store(serializer.copy(inputs.get(4)));
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.emitWindow(collector);
-
-		assertSetEquals(StreamWindow.fromElements(inputs.get(0), inputs.get(1)), collected.get(0));
-
-		wb.store(serializer.copy(inputs.get(4)));
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.store(serializer.copy(inputs.get(2)));
-		wb.emitWindow(collector);
-
-		assertSetEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(2, 0), inputs.get(1)), collected.get(1));
-
-
-	}
-
-	private static <T> void assertSetEquals(Collection<T> first, Collection<T> second) {
-		assertEquals(new HashSet<T>(first), new HashSet<T>(second));
-	}
-
-	@SuppressWarnings("serial")
-	private class Reducer implements ReduceFunction<Tuple2<Integer, Integer>> {
-
-		public List<Tuple2<Integer, Integer>> allInputs = new ArrayList<Tuple2<Integer, Integer>>();
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1,
-											   Tuple2<Integer, Integer> value2) throws Exception {
-			allInputs.add(value2);
-			value1.f0 = value1.f0 + value2.f0;
-			value1.f1 = value1.f1 + value2.f1;
-			return value1;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.java
deleted file mode 100644
index 2279264..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducerTest.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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class JumpingCountPreReducerTest {
-
-	TypeSerializer<Tuple2<Integer, Integer>> serializer = TypeExtractor.getForObject(
-			new Tuple2<Integer, Integer>(1, 1)).createSerializer(null);
-
-	Reducer reducer = new Reducer();
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testEmitWindow() throws Exception {
-
-		List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
-		inputs.add(new Tuple2<Integer, Integer>(1, 1));
-		inputs.add(new Tuple2<Integer, Integer>(2, 0));
-		inputs.add(new Tuple2<Integer, Integer>(3, -1));
-		inputs.add(new Tuple2<Integer, Integer>(4, -2));
-		inputs.add(new Tuple2<Integer, Integer>(5, -3));
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-		List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();
-
-		WindowBuffer<Tuple2<Integer, Integer>> wb = new JumpingCountPreReducer<Tuple2<Integer, Integer>>(
-				reducer, serializer, 2);
-
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.store(serializer.copy(inputs.get(2)));
-		wb.store(serializer.copy(inputs.get(3)));
-		wb.store(serializer.copy(inputs.get(4)));
-
-		wb.emitWindow(collector);
-
-		assertEquals(1, collected.size());
-		assertEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(12, -6)),
-				collected.get(0));
-
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.store(serializer.copy(inputs.get(2)));
-
-		// Nothing should happen here
-		wb.evict(3);
-
-		wb.store(serializer.copy(inputs.get(3)));
-
-		wb.emitWindow(collector);
-
-		assertEquals(2, collected.size());
-		assertEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(7, -3)),
-				collected.get(1));
-
-		// Test whether function is mutating inputs or not
-		assertEquals(3, reducer.allInputs.size());
-		assertEquals(reducer.allInputs.get(0), inputs.get(3));
-		assertEquals(reducer.allInputs.get(1), inputs.get(4));
-		assertEquals(reducer.allInputs.get(2), inputs.get(3));
-	}
-
-	@SuppressWarnings("serial")
-	private class Reducer implements ReduceFunction<Tuple2<Integer, Integer>> {
-
-		public List<Tuple2<Integer, Integer>> allInputs = new ArrayList<Tuple2<Integer, Integer>>();
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1,
-											   Tuple2<Integer, Integer> value2) throws Exception {
-			allInputs.add(value2);
-			value1.f0 = value1.f0 + value2.f0;
-			value1.f1 = value1.f1 + value2.f1;
-			return value1;
-		}
-
-	}
-
-}


[09/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
[FLINK-2780] Remove Old Windowing Logic and API

This rewrites the few examples and tests that are remaining using the
new Windowing API.


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

Branch: refs/heads/master
Commit: 0c1141abcd0f58dcf2ec9abcccd47c5e5410b074
Parents: b2b2781
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Sun Oct 4 10:33:06 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Fri Oct 9 11:15:59 2015 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    |  89 --
 .../api/datastream/DiscretizedStream.java       | 341 --------
 .../api/datastream/WindowedDataStream.java      | 867 -------------------
 .../api/functions/RichWindowMapFunction.java    |  40 -
 .../api/functions/WindowMapFunction.java        |  35 -
 .../windowing/delta/CosineDistance.java         |   2 +-
 .../windowing/delta/EuclideanDistance.java      |   2 +-
 .../delta/ExtractionAwareDeltaFunction.java     |   2 +-
 .../delta/extractor/ArrayFromTuple.java         |  74 ++
 .../delta/extractor/ConcatenatedExtract.java    |  68 ++
 .../windowing/delta/extractor/Extractor.java    |  43 +
 .../delta/extractor/FieldFromArray.java         |  59 ++
 .../delta/extractor/FieldFromTuple.java         |  58 ++
 .../delta/extractor/FieldsFromArray.java        |  67 ++
 .../delta/extractor/FieldsFromTuple.java        |  53 ++
 .../flink/streaming/api/graph/StreamGraph.java  |   4 -
 .../streaming/api/graph/WindowingOptimizer.java | 161 ----
 .../operators/windowing/EmptyWindowFilter.java  |  32 -
 .../windowing/GroupedActiveDiscretizer.java     | 129 ---
 .../windowing/GroupedStreamDiscretizer.java     | 132 ---
 .../windowing/GroupedWindowBuffer.java          |  69 --
 .../windowing/ParallelGroupedMerge.java         |  41 -
 .../api/operators/windowing/ParallelMerge.java  | 145 ----
 .../windowing/ParallelMergeOperator.java        |  43 -
 .../operators/windowing/StreamDiscretizer.java  | 237 -----
 .../operators/windowing/StreamWindowBuffer.java |  69 --
 .../operators/windowing/WindowFlattener.java    |  51 --
 .../api/operators/windowing/WindowFolder.java   | 100 ---
 .../api/operators/windowing/WindowMapper.java   |  94 --
 .../api/operators/windowing/WindowMerger.java   |  73 --
 .../windowing/WindowPartExtractor.java          |  55 --
 .../operators/windowing/WindowPartitioner.java  |  79 --
 .../api/operators/windowing/WindowReducer.java  |  99 ---
 .../streaming/api/windowing/StreamWindow.java   | 276 ------
 .../api/windowing/StreamWindowSerializer.java   | 148 ----
 .../api/windowing/StreamWindowTypeInfo.java     | 104 ---
 .../streaming/api/windowing/WindowEvent.java    |  71 --
 .../streaming/api/windowing/WindowUtils.java    | 203 -----
 .../api/windowing/evictors/TimeEvictor.java     |   3 +-
 .../api/windowing/extractor/ArrayFromTuple.java |  74 --
 .../extractor/ConcatenatedExtract.java          |  68 --
 .../api/windowing/extractor/Extractor.java      |  43 -
 .../api/windowing/extractor/FieldFromArray.java |  59 --
 .../api/windowing/extractor/FieldFromTuple.java |  58 --
 .../windowing/extractor/FieldsFromArray.java    |  67 --
 .../windowing/extractor/FieldsFromTuple.java    |  53 --
 .../streaming/api/windowing/helper/Count.java   | 101 ---
 .../streaming/api/windowing/helper/Delta.java   | 105 ---
 .../api/windowing/helper/FullStream.java        |  57 --
 .../api/windowing/helper/SystemTimestamp.java   |  37 -
 .../streaming/api/windowing/helper/Time.java    | 153 ----
 .../api/windowing/helper/Timestamp.java         |  39 -
 .../api/windowing/helper/TimestampWrapper.java  |  65 --
 .../api/windowing/helper/WindowingHelper.java   |  61 --
 .../ActiveCloneableEvictionPolicyWrapper.java   |  62 --
 .../windowing/policy/ActiveEvictionPolicy.java  |  52 --
 .../policy/ActiveEvictionPolicyWrapper.java     |  64 --
 .../windowing/policy/ActiveTriggerCallback.java |  45 -
 .../windowing/policy/ActiveTriggerPolicy.java   |  81 --
 .../windowing/policy/CentralActiveTrigger.java  |  45 -
 .../policy/CloneableEvictionPolicy.java         |  44 -
 .../policy/CloneableMultiEvictionPolicy.java    |  84 --
 .../policy/CloneableMultiTriggerPolicy.java     |  63 --
 .../policy/CloneableTriggerPolicy.java          |  44 -
 .../windowing/policy/CountEvictionPolicy.java   | 155 ----
 .../windowing/policy/CountTriggerPolicy.java    | 117 ---
 .../api/windowing/policy/DeltaPolicy.java       | 167 ----
 .../api/windowing/policy/EvictionPolicy.java    |  47 -
 .../windowing/policy/KeepAllEvictionPolicy.java |  29 -
 .../windowing/policy/MultiEvictionPolicy.java   | 170 ----
 .../windowing/policy/MultiTriggerPolicy.java    | 123 ---
 .../api/windowing/policy/PunctuationPolicy.java | 147 ----
 .../windowing/policy/TimeEvictionPolicy.java    | 167 ----
 .../api/windowing/policy/TimeTriggerPolicy.java | 209 -----
 .../api/windowing/policy/TriggerPolicy.java     |  54 --
 .../policy/TumblingEvictionPolicy.java          | 104 ---
 .../windowbuffer/BasicWindowBuffer.java         |  73 --
 .../JumpingCountGroupedPreReducer.java          |  54 --
 .../windowbuffer/JumpingCountPreReducer.java    |  56 --
 .../JumpingTimeGroupedPreReducer.java           |  56 --
 .../windowbuffer/JumpingTimePreReducer.java     |  58 --
 .../windowing/windowbuffer/PreAggregator.java   |  27 -
 .../SlidingCountGroupedPreReducer.java          |  83 --
 .../windowbuffer/SlidingCountPreReducer.java    |  81 --
 .../windowbuffer/SlidingGroupedPreReducer.java  | 153 ----
 .../windowbuffer/SlidingPreReducer.java         | 175 ----
 .../SlidingTimeGroupedPreReducer.java           | 102 ---
 .../windowbuffer/SlidingTimePreReducer.java     | 102 ---
 .../windowbuffer/TumblingGroupedPreReducer.java | 107 ---
 .../windowbuffer/TumblingPreReducer.java        |  98 ---
 .../windowing/windowbuffer/WindowBuffer.java    |  61 --
 .../flink/streaming/api/DataStreamTest.java     | 125 +--
 .../api/complex/ComplexIntegrationTest.java     | 198 +++--
 .../delta/extractor/ArrayFromTupleTest.java     | 118 +++
 .../extractor/ConcatenatedExtractTest.java      |  77 ++
 .../delta/extractor/FieldFromArrayTest.java     |  54 ++
 .../delta/extractor/FieldFromTupleTest.java     |  83 ++
 .../delta/extractor/FieldsFromArrayTest.java    | 108 +++
 .../delta/extractor/FieldsFromTupleTest.java    | 106 +++
 .../api/operators/co/SelfConnectionTest.java    |  18 -
 .../windowing/GroupedStreamDiscretizerTest.java | 104 ---
 .../windowing/ParallelMergeITCase.java          | 101 ---
 .../operators/windowing/ParallelMergeTest.java  | 122 ---
 .../windowing/StreamDiscretizerTest.java        | 117 ---
 .../windowing/WindowFlattenerTest.java          |  53 --
 .../operators/windowing/WindowFolderTest.java   |  61 --
 .../operators/windowing/WindowMapperTest.java   |  60 --
 .../operators/windowing/WindowMergerTest.java   |  75 --
 .../windowing/WindowPartitionerTest.java        |  75 --
 .../operators/windowing/WindowReducerTest.java  |  61 --
 .../operators/windowing/WindowingITCase.java    | 529 -----------
 .../api/windowing/StreamWindowTest.java         | 201 -----
 .../api/windowing/StreamWindowTypeInfoTest.java |  51 --
 .../windowing/extractor/ArrayFromTupleTest.java | 118 ---
 .../extractor/ConcatenatedExtractTest.java      |  77 --
 .../windowing/extractor/FieldFromArrayTest.java |  55 --
 .../windowing/extractor/FieldFromTupleTest.java |  84 --
 .../extractor/FieldsFromArrayTest.java          | 108 ---
 .../extractor/FieldsFromTupleTest.java          | 106 ---
 .../policy/CountEvictionPolicyTest.java         | 136 ---
 .../policy/CountTriggerPolicyTest.java          | 109 ---
 .../api/windowing/policy/DeltaPolicyTest.java   |  88 --
 .../policy/MultiEvictionPolicyTest.java         | 186 ----
 .../policy/MultiTriggerPolicyTest.java          | 245 ------
 .../windowing/policy/PunctuationPolicyTest.java | 155 ----
 .../policy/TimeEvictionPolicyTest.java          | 164 ----
 .../windowing/policy/TimeTriggerPolicyTest.java | 156 ----
 .../policy/TumblingEvictionPolicyTest.java      |  43 -
 .../windowbuffer/BasicWindowBufferTest.java     |  86 --
 .../JumpingCountGroupedPreReducerTest.java      | 157 ----
 .../JumpingCountPreReducerTest.java             | 107 ---
 .../windowbuffer/JumpingTimePreReducerTest.java |  96 --
 .../SlidingCountGroupedPreReducerTest.java      | 235 -----
 .../SlidingCountPreReducerTest.java             | 216 -----
 .../SlidingTimeGroupedPreReducerTest.java       | 387 ---------
 .../windowbuffer/SlidingTimePreReducerTest.java | 324 -------
 .../TumblingGroupedPreReducerTest.java          | 151 ----
 .../windowbuffer/TumblingPreReducerTest.java    | 104 ---
 .../ml/IncrementalLearningSkeleton.java         |  48 +-
 .../util/IncrementalLearningSkeletonData.java   |  16 +-
 .../examples/windowing/SessionWindowing.java    |  84 +-
 .../examples/windowing/TopSpeedWindowing.java   |  49 +-
 .../examples/windowing/WindowWordCount.java     |  13 +-
 .../util/TopSpeedWindowingExampleData.java      |  96 +-
 .../examples/windowing/TopSpeedWindowing.scala  |  28 +-
 .../streaming/api/scala/AllWindowedStream.scala |  25 +
 .../flink/streaming/api/scala/DataStream.scala  |  36 +-
 .../api/scala/WindowedDataStream.scala          | 338 --------
 .../streaming/api/scala/WindowedStream.scala    |  25 +
 .../flink/streaming/api/scala/package.scala     |   7 +-
 .../streaming/api/scala/windowing/Delta.scala   |  46 -
 .../streaming/api/scala/windowing/Time.scala    |  53 --
 .../streaming/api/scala/DataStreamTest.scala    |  51 +-
 .../StreamingScalaAPICompletenessTest.scala     |   7 +-
 154 files changed, 1440 insertions(+), 14289 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index ee8b3d2..80e0e47 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -62,13 +62,6 @@ import org.apache.flink.streaming.api.transformations.UnionTransformation;
 import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
 import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.api.windowing.helper.Count;
-import org.apache.flink.streaming.api.windowing.helper.Delta;
-import org.apache.flink.streaming.api.windowing.helper.FullStream;
-import org.apache.flink.streaming.api.windowing.helper.Time;
-import org.apache.flink.streaming.api.windowing.helper.WindowingHelper;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
 import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.api.windowing.windows.Window;
@@ -630,88 +623,6 @@ public class DataStream<T> {
 	}
 
 	/**
-	 * Create a {@link WindowedDataStream} that can be used to apply
-	 * transformation like {@link WindowedDataStream#reduceWindow},
-	 * {@link WindowedDataStream#mapWindow} or aggregations on preset
-	 * chunks(windows) of the data stream. To define windows a
-	 * {@link WindowingHelper} such as {@link Time}, {@link Count},
-	 * {@link Delta} and {@link FullStream} can be used.
-	 *
-	 * <p>
-	 * When applied to a grouped data stream, the windows (evictions) and slide sizes
-	 * (triggers) will be computed on a per group basis.
-	 *
-	 * <p>
-	 * For more advanced control over the trigger and eviction policies please refer to
-	 * {@link #window(TriggerPolicy, EvictionPolicy)}
-	 *
-	 * <p>
-	 * For example, to create a sum every 5 seconds in a tumbling fashion:
-	 *
-	 * <pre>
-	 * {@code ds.window(Time.of(5, TimeUnit.SECONDS)).sum(field)}
-	 * </pre>
-	 *
-	 * <p>
-	 * To create sliding windows use the
-	 * {@link WindowedDataStream#every(WindowingHelper)}, for example with 3 second slides:</br>
-	 *
-	 * <pre>
-	 * 
-	 * {@code
-	 * ds.window(Time.of(5, TimeUnit.SECONDS)).every(Time.of(3, TimeUnit.SECONDS)).sum(field)
-	 * }
-	 *
-	 * </pre>
-	 * 
-	 * @param policyHelper
-	 *            Any {@link WindowingHelper} such as {@link Time},
-	 *            {@link Count}, {@link Delta} {@link FullStream} to define the
-	 *            window size.
-	 *
-	 * @return A {@link WindowedDataStream} providing further operations.
-	 */
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	public WindowedDataStream<T> window(WindowingHelper policyHelper) {
-		policyHelper.setExecutionConfig(getExecutionConfig());
-		return new WindowedDataStream<T>(this, policyHelper);
-	}
-
-	/**
-	 * Create a {@link WindowedDataStream} using the given {@link TriggerPolicy}
-	 * and {@link EvictionPolicy}. Windowing can be used to apply transformation
-	 * like {@link WindowedDataStream#reduceWindow},
-	 * {@link WindowedDataStream#mapWindow} or aggregations on preset
-	 * chunks(windows) of the data stream.
-	 *
-	 * <p>
-	 * For most common use-cases please refer to {@link #window(WindowingHelper)}
-	 * 
-	 * @param trigger
-	 *            The {@link TriggerPolicy} that will determine how often the
-	 *            user function is called on the window.
-	 * @param eviction
-	 *            The {@link EvictionPolicy} that will determine the number of
-	 *            elements in each time window.
-	 * @return A {@link WindowedDataStream} providing further operations.
-	 */
-	public WindowedDataStream<T> window(TriggerPolicy<T> trigger, EvictionPolicy<T> eviction) {
-		return new WindowedDataStream<T>(this, trigger, eviction);
-	}
-
-	/**
-	 * Create a {@link WindowedDataStream} on the full stream history, to
-	 * produce periodic aggregates.
-	 * 
-	 * @return A {@link WindowedDataStream} providing further operations.
-	 */
-	@SuppressWarnings("rawtypes")
-	public WindowedDataStream<T> every(WindowingHelper policyHelper) {
-		policyHelper.setExecutionConfig(getExecutionConfig());
-		return window(FullStream.window()).every(policyHelper);
-	}
-
-	/**
 	 * Windows this {@code DataStream} into tumbling time windows.
 	 *
 	 * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
deleted file mode 100644
index 18c2cee..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DiscretizedStream.java
+++ /dev/null
@@ -1,341 +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.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamFilter;
-import org.apache.flink.streaming.api.operators.StreamFlatMap;
-import org.apache.flink.streaming.api.operators.windowing.EmptyWindowFilter;
-import org.apache.flink.streaming.api.operators.windowing.ParallelGroupedMerge;
-import org.apache.flink.streaming.api.operators.windowing.ParallelMerge;
-import org.apache.flink.streaming.api.operators.windowing.ParallelMergeOperator;
-import org.apache.flink.streaming.api.operators.windowing.WindowFlattener;
-import org.apache.flink.streaming.api.operators.windowing.WindowFolder;
-import org.apache.flink.streaming.api.operators.windowing.WindowMapper;
-import org.apache.flink.streaming.api.operators.windowing.WindowMerger;
-import org.apache.flink.streaming.api.operators.windowing.WindowPartExtractor;
-import org.apache.flink.streaming.api.operators.windowing.WindowPartitioner;
-import org.apache.flink.streaming.api.operators.windowing.WindowReducer;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.StreamWindowTypeInfo;
-import org.apache.flink.streaming.api.windowing.WindowUtils.WindowKey;
-import org.apache.flink.streaming.api.windowing.WindowUtils.WindowTransformation;
-
-/**
- * A {@link DiscretizedStream} represents a data stream that has been divided
- * into windows (predefined chunks). User defined function such as
- * {@link #reduceWindow(ReduceFunction)}, {@link #mapWindow(WindowMapFunction)},
- * or aggregations can be applied to the windows.
- * 
- * @param <OUT>
- *            The output type of the {@link DiscretizedStream}
- */
-public class DiscretizedStream<OUT> extends WindowedDataStream<OUT> {
-
-	private SingleOutputStreamOperator<StreamWindow<OUT>, ?> discretizedStream;
-	private WindowTransformation transformation;
-	protected boolean isPartitioned = false;
-
-	protected DiscretizedStream(SingleOutputStreamOperator<StreamWindow<OUT>, ?> discretizedStream,
-			KeySelector<OUT, ?> keyByKey, WindowTransformation tranformation,
-			boolean isPartitioned) {
-		super();
-		this.keyByKey = keyByKey;
-		this.discretizedStream = discretizedStream;
-		this.transformation = tranformation;
-		this.isPartitioned = isPartitioned;
-	}
-
-	/**
-	 * Gets the name of the current data stream. This name is
-	 * used by the visualization and logging during runtime.
-	 *
-	 * @return Name of the stream.
-	 */
-	public String getName(){
-		return discretizedStream.getName();
-	}
-
-	/**
-	 * Sets the name of the current data stream. This name is
-	 * used by the visualization and logging during runtime.
-	 *
-	 * @return The named operator.
-	 */
-	public DiscretizedStream<OUT> name(String name){
-		discretizedStream.name(name);
-		return this;
-	}
-
-	public DataStream<OUT> flatten() {
-		return discretizedStream.transform("Window Flatten", getType(), new WindowFlattener<OUT>()).setParallelism(discretizedStream.getParallelism());
-	}
-
-	public DataStream<StreamWindow<OUT>> getDiscretizedStream() {
-		return discretizedStream;
-	}
-
-	@Override
-	public DiscretizedStream<OUT> reduceWindow(ReduceFunction<OUT> reduceFunction) {
-
-		DiscretizedStream<OUT> out = partition(transformation).transform(
-				WindowTransformation.REDUCEWINDOW, "Window Reduce", getType(),
-				new WindowReducer<OUT>(reduceFunction)).merge();
-
-		// If we merged a non-grouped reduce transformation we need to reduce
-		// again
-		if (!isGrouped() && ((OneInputTransformation<?, ?>)out.discretizedStream.getTransformation()).getOperator() instanceof WindowMerger) {
-			return out.transform(WindowTransformation.REDUCEWINDOW, "Window Reduce", out.getType(),
-					new WindowReducer<OUT>(discretizedStream.clean(reduceFunction)));
-		} else {
-			return out;
-		}
-	}
-
-	/**
-	 * This method implements the parallel time reduce logic for time windows
-	 * 
-	 * @param reduceFunction
-	 *            The reduce function to be applied on the windows
-	 * @return The reduced DataStream
-	 */
-	protected DiscretizedStream<OUT> timeReduce(ReduceFunction<OUT> reduceFunction) {
-
-		// Since we also emit the empty windows for bookkeeping, we need to
-		// filter them out
-		DiscretizedStream<OUT> nonEmpty = filterEmpty(this);
-
-		// We extract the number of parts from each window we will merge using
-		// this afterwards
-		DataStream<Tuple2<Integer, Integer>> numOfParts = extractPartsByID(this);
-
-		// We merge the windows by the number of parts
-		return wrap(parallelMerge(numOfParts, nonEmpty, reduceFunction), false);
-
-	}
-
-	private SingleOutputStreamOperator<StreamWindow<OUT>, ?> parallelMerge(
-			DataStream<Tuple2<Integer, Integer>> numOfParts, DiscretizedStream<OUT> reduced,
-			ReduceFunction<OUT> reduceFunction) {
-
-		ParallelMerge<OUT> parallelMerger = isGrouped() ? new ParallelGroupedMerge<OUT>()
-				: new ParallelMerge<OUT>(reduceFunction);
-
-		return reduced.discretizedStream
-				.keyBy(new WindowKey<OUT>())
-				.connect(numOfParts.keyBy(0))
-				.transform(
-						"CoFlatMap",
-						reduced.discretizedStream.getType(),
-						new ParallelMergeOperator<OUT>(parallelMerger));
-	}
-
-	@Override
-	public <R> DiscretizedStream<R> mapWindow(WindowMapFunction<OUT, R> windowMapFunction) {
-
-		TypeInformation<R> retType = getWindowMapReturnTypes(windowMapFunction, getType());
-
-		return mapWindow(windowMapFunction, retType);
-	}
-
-	@Override
-	public <R> DiscretizedStream<R> mapWindow(WindowMapFunction<OUT, R> windowMapFunction,
-			TypeInformation<R> returnType) {
-		DiscretizedStream<R> out = partition(transformation).transform(
-				WindowTransformation.MAPWINDOW, "Window Map", returnType,
-				new WindowMapper<OUT, R>(discretizedStream.clean(windowMapFunction))).setParallelism(discretizedStream.getParallelism()).merge();
-
-		return out;
-	}
-
-	@Override
-	public <R> DiscretizedStream<R> foldWindow(R initialValue, FoldFunction<OUT, R> foldFunction,
-			TypeInformation<R> outType) {
-
-		DiscretizedStream<R> out = partition(transformation).transform(
-				WindowTransformation.FOLDWINDOW, "Fold Window", outType,
-				new WindowFolder<OUT, R>(discretizedStream.clean(foldFunction), initialValue))
-				.setParallelism(discretizedStream.getParallelism())
-				.merge();
-		return out;
-	}
-
-	private <R> DiscretizedStream<R> transform(WindowTransformation transformation,
-			String operatorName, TypeInformation<R> retType,
-			OneInputStreamOperator<StreamWindow<OUT>, StreamWindow<R>> operator) {
-
-		return wrap(discretizedStream.transform(operatorName, new StreamWindowTypeInfo<R>(retType),
-				operator).setParallelism(discretizedStream.getParallelism()), transformation);
-	}
-
-	private DiscretizedStream<OUT> filterEmpty(DiscretizedStream<OUT> input) {
-		StreamFilter<StreamWindow<OUT>> emptyFilter = new StreamFilter<StreamWindow<OUT>>(new EmptyWindowFilter<OUT>());
-		emptyFilter.disableInputCopy();
-		return wrap(input.discretizedStream.transform("Filter", input.discretizedStream.getType(), emptyFilter), input.isPartitioned);
-	}
-
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	private DataStream<Tuple2<Integer, Integer>> extractPartsByID(DiscretizedStream<OUT> input) {
-		StreamFlatMap<StreamWindow<OUT>, Tuple2<Integer, Integer>> partExtractor = new StreamFlatMap<StreamWindow<OUT>, Tuple2<Integer, Integer>>(
-				new WindowPartExtractor<OUT>());
-		partExtractor.disableInputCopy();
-		return input.discretizedStream.transform("ExtractParts", new TupleTypeInfo(Tuple2.class,
-				BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO), partExtractor);
-	}
-
-	private DiscretizedStream<OUT> partition(WindowTransformation transformation) {
-
-		int parallelism = discretizedStream.getParallelism();
-
-		if (isGrouped()) {
-			DiscretizedStream<OUT> out = transform(transformation, "Window partitioner", getType(),
-					new WindowPartitioner<OUT>(keyByKey)).setParallelism(parallelism);
-
-			out.keyByKey = null;
-			out.isPartitioned = true;
-
-			return out;
-		} else if (transformation == WindowTransformation.REDUCEWINDOW
-				&& parallelism != discretizedStream.getExecutionEnvironment().getParallelism()) {
-			DiscretizedStream<OUT> out = transform(transformation, "Window partitioner", getType(),
-					new WindowPartitioner<OUT>(parallelism)).setParallelism(parallelism);
-
-			out.isPartitioned = true;
-
-			return out;
-		} else {
-			return this;
-		}
-	}
-
-	private DiscretizedStream<OUT> setParallelism(int parallelism) {
-		return wrap(discretizedStream.setParallelism(parallelism), isPartitioned);
-	}
-
-	private DiscretizedStream<OUT> merge() {
-		TypeInformation<StreamWindow<OUT>> type = discretizedStream.getType();
-
-		// Only merge partitioned streams
-		if (isPartitioned) {
-			return wrap(
-					discretizedStream.keyBy(new WindowKey<OUT>()).transform("Window Merger",
-							type, new WindowMerger<OUT>()).setParallelism(discretizedStream.getParallelism()), false);
-		} else {
-			return this;
-		}
-
-	}
-
-	@SuppressWarnings("unchecked")
-	private <R> DiscretizedStream<R> wrap(SingleOutputStreamOperator<StreamWindow<R>, ?> stream,
-			boolean isPartitioned) {
-		return new DiscretizedStream<R>(stream, (KeySelector<R, ?>) this.keyByKey,
-				transformation, isPartitioned);
-	}
-
-	@SuppressWarnings("unchecked")
-	private <R> DiscretizedStream<R> wrap(SingleOutputStreamOperator<StreamWindow<R>, ?> stream,
-			WindowTransformation transformation) {
-		return new DiscretizedStream<R>(stream, (KeySelector<R, ?>) this.keyByKey,
-				transformation, isPartitioned);
-	}
-
-	@SuppressWarnings("rawtypes")
-	protected Class<?> getClassAtPos(int pos) {
-		Class<?> type;
-		TypeInformation<OUT> outTypeInfo = getType();
-		if (outTypeInfo.isTupleType()) {
-			type = ((TupleTypeInfo) outTypeInfo).getTypeAt(pos).getTypeClass();
-
-		} else if (outTypeInfo instanceof BasicArrayTypeInfo) {
-
-			type = ((BasicArrayTypeInfo) outTypeInfo).getComponentTypeClass();
-
-		} else if (outTypeInfo instanceof PrimitiveArrayTypeInfo) {
-			Class<?> clazz = outTypeInfo.getTypeClass();
-			if (clazz == boolean[].class) {
-				type = Boolean.class;
-			} else if (clazz == short[].class) {
-				type = Short.class;
-			} else if (clazz == int[].class) {
-				type = Integer.class;
-			} else if (clazz == long[].class) {
-				type = Long.class;
-			} else if (clazz == float[].class) {
-				type = Float.class;
-			} else if (clazz == double[].class) {
-				type = Double.class;
-			} else if (clazz == char[].class) {
-				type = Character.class;
-			} else {
-				throw new IndexOutOfBoundsException("Type could not be determined for array");
-			}
-
-		} else if (pos == 0) {
-			type = outTypeInfo.getTypeClass();
-		} else {
-			throw new IndexOutOfBoundsException("Position is out of range");
-		}
-		return type;
-	}
-
-	@Override
-	public ExecutionConfig getExecutionConfig() {
-		return discretizedStream.getExecutionConfig();
-	}
-
-	/**
-	 * Gets the output type.
-	 * 
-	 * @return The output type.
-	 */
-	public TypeInformation<OUT> getType() {
-		return ((StreamWindowTypeInfo<OUT>) discretizedStream.getType()).getInnerType();
-	}
-
-	private static <IN, OUT> TypeInformation<OUT> getWindowMapReturnTypes(
-			WindowMapFunction<IN, OUT> windowMapInterface, TypeInformation<IN> inType) {
-		return TypeExtractor.getUnaryOperatorReturnType((Function) windowMapInterface,
-				WindowMapFunction.class, true, true, inType, null, false);
-	}
-
-	protected DiscretizedStream<OUT> copy() {
-		return new DiscretizedStream<OUT>(discretizedStream, keyByKey, transformation, isPartitioned);
-	}
-
-	@Override
-	public WindowedDataStream<OUT> local() {
-		throw new UnsupportedOperationException(
-				"Local discretisation can only be applied after defining the discretisation logic");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
deleted file mode 100644
index c1c5f6d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedDataStream.java
+++ /dev/null
@@ -1,867 +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.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichFoldFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.RichWindowMapFunction;
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType;
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.windowing.GroupedActiveDiscretizer;
-import org.apache.flink.streaming.api.operators.windowing.GroupedStreamDiscretizer;
-import org.apache.flink.streaming.api.operators.windowing.GroupedWindowBuffer;
-import org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer;
-import org.apache.flink.streaming.api.operators.windowing.StreamWindowBuffer;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.StreamWindowTypeInfo;
-import org.apache.flink.streaming.api.windowing.WindowEvent;
-import org.apache.flink.streaming.api.windowing.WindowUtils;
-import org.apache.flink.streaming.api.windowing.WindowUtils.WindowTransformation;
-import org.apache.flink.streaming.api.windowing.helper.Time;
-import org.apache.flink.streaming.api.windowing.helper.WindowingHelper;
-import org.apache.flink.streaming.api.windowing.policy.CentralActiveTrigger;
-import org.apache.flink.streaming.api.windowing.policy.CloneableEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.CloneableTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.CountTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.KeepAllEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TumblingEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBuffer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.JumpingCountGroupedPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.JumpingCountPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.JumpingTimeGroupedPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.JumpingTimePreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.PreAggregator;
-import org.apache.flink.streaming.api.windowing.windowbuffer.SlidingCountGroupedPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.SlidingCountPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.SlidingTimeGroupedPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.SlidingTimePreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.TumblingGroupedPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.TumblingPreReducer;
-import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-
-/**
- * A {@link WindowedDataStream} represents a data stream that has been
- * discretised into windows. User defined function such as
- * {@link #reduceWindow(ReduceFunction)}, {@link #mapWindow(WindowMapFunction)} or aggregations
- * can be applied to the windows. The results of these transformations are also
- * WindowedDataStreams of the same discretisation unit.
- * 
- * @param <T> The output type of the {@link WindowedDataStream}
- */
-public class WindowedDataStream<T> {
-
-	protected DataStream<T> dataStream;
-
-	protected boolean isLocal = false;
-
-	protected KeySelector<T, ?> discretizerKey;
-	protected KeySelector<T, ?> keyByKey;
-
-	protected WindowingHelper<T> triggerHelper;
-	protected WindowingHelper<T> evictionHelper;
-
-	protected TriggerPolicy<T> userTrigger;
-	protected EvictionPolicy<T> userEvicter;
-
-	protected WindowedDataStream(DataStream<T> dataStream, WindowingHelper<T> policyHelper) {
-		this.dataStream = dataStream;
-		this.triggerHelper = policyHelper;
-
-		if (dataStream instanceof KeyedStream) {
-			this.discretizerKey = ((KeyedStream<T, ?>) dataStream).keySelector;
-		}
-	}
-
-	protected WindowedDataStream(DataStream<T> dataStream, TriggerPolicy<T> trigger,
-			EvictionPolicy<T> evicter) {
-		this.dataStream = dataStream;
-
-		this.userTrigger = trigger;
-		this.userEvicter = evicter;
-
-		if (dataStream instanceof KeyedStream) {
-			this.discretizerKey = ((KeyedStream<T, ?>) dataStream).keySelector;
-		}
-	}
-
-	protected WindowedDataStream(WindowedDataStream<T> windowedDataStream) {
-		this.dataStream = windowedDataStream.dataStream;
-		this.discretizerKey = windowedDataStream.discretizerKey;
-		this.keyByKey = windowedDataStream.keyByKey;
-		this.triggerHelper = windowedDataStream.triggerHelper;
-		this.evictionHelper = windowedDataStream.evictionHelper;
-		this.userTrigger = windowedDataStream.userTrigger;
-		this.userEvicter = windowedDataStream.userEvicter;
-		this.isLocal = windowedDataStream.isLocal;
-	}
-
-	public WindowedDataStream() {
-	}
-
-	/**
-	 * Defines the slide size (trigger frequency) for the windowed data stream.
-	 * This controls how often the user defined function will be triggered on
-	 * the window. </br></br> For example to get a window of 5 elements with a
-	 * slide of 2 seconds use: </br></br>
-	 * {@code ds.window(Count.of(5)).every(Time.of(2,TimeUnit.SECONDS))}
-	 * </br></br> The user function in this case will be called on the 5 most
-	 * recent elements every 2 seconds
-	 * 
-	 * @param policyHelper
-	 *            The policy that define the triggering frequency
-	 * 
-	 * @return The windowed data stream with triggering set
-	 */
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	public WindowedDataStream<T> every(WindowingHelper policyHelper) {
-		policyHelper.setExecutionConfig(getExecutionConfig());
-		WindowedDataStream<T> ret = this.copy();
-		if (ret.evictionHelper == null) {
-			ret.evictionHelper = ret.triggerHelper;
-			ret.triggerHelper = policyHelper;
-		}
-
-		return ret;
-	}
-
-	/**
-	 * Groups the elements of the {@link WindowedDataStream} by the given key
-	 * positions. The window sizes (evictions) and slide sizes (triggers) will
-	 * be calculated on the whole stream (in a global fashion), but the user
-	 * defined functions will be applied on a per group basis. </br></br> To get
-	 * windows and triggers on a per group basis apply the
-	 * {@link DataStream#window} operator on an already grouped data stream.
-	 * 
-	 * @param fields
-	 *            The position of the fields to group by.
-	 * @return The grouped {@link WindowedDataStream}
-	 */
-	public WindowedDataStream<T> keyBy(int... fields) {
-		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return keyBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
-		} else {
-			return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
-		}
-	}
-
-	/**
-	 * Groups the elements of the {@link WindowedDataStream} by the given field
-	 * expressions. The window sizes (evictions) and slide sizes (triggers) will
-	 * be calculated on the whole stream (in a global fashion), but the user
-	 * defined functions will be applied on a per group basis. </br></br> To get
-	 * windows and triggers on a per group basis apply the
-	 * {@link DataStream#window} operator on an already grouped data stream.
-	 * </br></br> A field expression is either the name of a public field or a
-	 * getter method with parentheses of the stream's underlying type. A dot can
-	 * be used to drill down into objects, as in
-	 * {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param fields
-	 *            The fields to group by
-	 * @return The grouped {@link WindowedDataStream}
-	 */
-	public WindowedDataStream<T> keyBy(String... fields) {
-		return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
-	}
-
-	/**
-	 * Groups the elements of the {@link WindowedDataStream} using the given
-	 * {@link KeySelector}. The window sizes (evictions) and slide sizes
-	 * (triggers) will be calculated on the whole stream (in a global fashion),
-	 * but the user defined functions will be applied on a per group basis.
-	 * </br></br> To get windows and triggers on a per group basis apply the
-	 * {@link DataStream#window} operator on an already grouped data stream.
-	 * 
-	 * @param keySelector
-	 *            The keySelector used to extract the key for grouping.
-	 * @return The grouped {@link WindowedDataStream}
-	 */
-	public WindowedDataStream<T> keyBy(KeySelector<T, ?> keySelector) {
-		WindowedDataStream<T> ret = this.copy();
-		ret.keyByKey = keySelector;
-		return ret;
-	}
-
-	private WindowedDataStream<T> keyBy(Keys<T> keys) {
-		return keyBy(clean(KeySelectorUtil.getSelectorForKeys(keys, getType(),
-				getExecutionConfig())));
-	}
-
-	/**
-	 * Sets the window discretisation local, meaning that windows will be
-	 * created in parallel at environment parallelism.
-	 * 
-	 * @return The WindowedDataStream with local discretisation
-	 */
-	public WindowedDataStream<T> local() {
-		WindowedDataStream<T> out = copy();
-		out.isLocal = true;
-		return out;
-	}
-
-	/**
-	 * Returns the {@link DataStream} of {@link StreamWindow}s which represent
-	 * the discretised stream. There is no ordering guarantee for the received
-	 * windows.
-	 * 
-	 * @return The discretised stream
-	 */
-	public DataStream<StreamWindow<T>> getDiscretizedStream() {
-		if (getEviction() instanceof KeepAllEvictionPolicy) {
-			throw new RuntimeException("Cannot get discretized stream for full stream window");
-		}
-		return discretize(WindowTransformation.NONE, new BasicWindowBuffer<T>())
-				.getDiscretizedStream();
-	}
-
-	/**
-	 * Flattens the results of the window computations and streams out the
-	 * window elements.
-	 * 
-	 * @return The data stream consisting of the individual records.
-	 */
-	public DataStream<T> flatten() {
-		return dataStream;
-	}
-
-	/**
-	 * Applies a reduce transformation on the windowed data stream by reducing
-	 * the current window at every trigger.The user can also extend the
-	 * {@link RichReduceFunction} to gain access to other features provided by
-	 * the {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * 
-	 * @param reduceFunction
-	 *            The reduce function that will be applied to the windows.
-	 * @return The transformed DataStream
-	 */
-	public DiscretizedStream<T> reduceWindow(ReduceFunction<T> reduceFunction) {
-
-		// We check whether we should apply parallel time discretization, which
-		// is a more complex exploiting the monotonic properties of time
-		// policies
-		if (WindowUtils.isTimeOnly(getTrigger(), getEviction()) && discretizerKey == null
-				&& dataStream.getParallelism() > 1) {
-			return timeReduce(reduceFunction);
-		} else {
-			WindowTransformation transformation = WindowTransformation.REDUCEWINDOW
-					.with(clean(reduceFunction));
-
-			WindowBuffer<T> windowBuffer = getWindowBuffer(transformation);
-
-			DiscretizedStream<T> discretized = discretize(transformation, windowBuffer);
-
-			if (windowBuffer instanceof PreAggregator) {
-				return discretized;
-			} else {
-				return discretized.reduceWindow(reduceFunction);
-			}
-		}
-	}
-
-	/**
-	 * Applies a fold transformation on the windowed data stream by folding the
-	 * current window at every trigger.The user can also extend the
-	 * {@link RichFoldFunction} to gain access to other features provided by the
-	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * This version of foldWindow uses user supplied typeinformation for
-	 * serializaton. Use this only when the system is unable to detect type
-	 * information.
-	 * 
-	 * @param foldFunction
-	 *            The fold function that will be applied to the windows.
-	 * @param initialValue
-	 *            Initial value given to foldFunction
-	 * @param outType
-	 *            The output type of the operator
-	 * @return The transformed DataStream
-	 */
-	public <R> DiscretizedStream<R> foldWindow(R initialValue, FoldFunction<T, R> foldFunction,
-			TypeInformation<R> outType) {
-
-		return discretize(WindowTransformation.FOLDWINDOW.with(clean(foldFunction)),
-				new BasicWindowBuffer<T>()).foldWindow(initialValue, foldFunction, outType);
-
-	}
-
-	/**
-	 * Applies a fold transformation on the windowed data stream by folding the
-	 * current window at every trigger.The user can also extend the
-	 * {@link RichFoldFunction} to gain access to other features provided by the
-	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * 
-	 * @param foldFunction
-	 *            The fold function that will be applied to the windows.
-	 * @param initialValue
-	 *            Initial value given to foldFunction
-	 * @return The transformed DataStream
-	 */
-	public <R> DiscretizedStream<R> foldWindow(R initialValue, FoldFunction<T, R> foldFunction) {
-
-		TypeInformation<R> outType = TypeExtractor.getFoldReturnTypes(clean(foldFunction),
-				getType());
-		return foldWindow(initialValue, foldFunction, outType);
-	}
-
-	/**
-	 * Applies a mapWindow transformation on the windowed data stream by calling
-	 * the mapWindow function on the window at every trigger. In contrast with
-	 * the standard binary reducer, with mapWindow allows the user to access all
-	 * elements of the window at the same time through the iterable interface.
-	 * The user can also extend the {@link RichWindowMapFunction} to gain access
-	 * to other features provided by the
-	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * 
-	 * @param windowMapFunction
-	 *            The function that will be applied to the windows.
-	 * @return The transformed DataStream
-	 */
-	public <R> DiscretizedStream<R> mapWindow(WindowMapFunction<T, R> windowMapFunction) {
-		return discretize(WindowTransformation.MAPWINDOW.with(clean(windowMapFunction)),
-				getWindowBuffer(WindowTransformation.MAPWINDOW)).mapWindow(windowMapFunction);
-	}
-
-	/**
-	 * Applies a mapWindow transformation on the windowed data stream by calling
-	 * the mapWindow function on the window at every trigger. In contrast with
-	 * the standard binary reducer, with mapWindow allows the user to access all
-	 * elements of the window at the same time through the iterable interface.
-	 * The user can also extend the {@link RichWindowMapFunction} to gain access
-	 * to other features provided by the
-	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * </br> </br> This version of mapWindow uses user supplied typeinformation
-	 * for serializaton. Use this only when the system is unable to detect type
-	 * information.
-	 * 
-	 * @param windowMapFunction
-	 *            The function that will be applied to the windows.
-	 * @param outType
-	 *            The output type of the operator.
-	 * @return The transformed DataStream
-	 */
-	public <R> DiscretizedStream<R> mapWindow(WindowMapFunction<T, R> windowMapFunction,
-			TypeInformation<R> outType) {
-
-		return discretize(WindowTransformation.MAPWINDOW.with(windowMapFunction),
-				getWindowBuffer(WindowTransformation.MAPWINDOW)).mapWindow(windowMapFunction,
-				outType);
-	}
-
-	private DiscretizedStream<T> discretize(WindowTransformation transformation,
-			WindowBuffer<T> windowBuffer) {
-
-		OneInputStreamOperator<T, WindowEvent<T>> discretizer = getDiscretizer();
-
-		OneInputStreamOperator<WindowEvent<T>, StreamWindow<T>> bufferOperator = getBufferOperator(windowBuffer);
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		TypeInformation<WindowEvent<T>> bufferEventType = new TupleTypeInfo(WindowEvent.class,
-				getType(), BasicTypeInfo.INT_TYPE_INFO);
-
-		int parallelism = getDiscretizerParallelism(transformation);
-
-		return new DiscretizedStream<T>(dataStream
-				.transform(discretizer.getClass().getSimpleName(), bufferEventType, discretizer)
-				.setParallelism(parallelism)
-				.transform(windowBuffer.getClass().getSimpleName(),
-						new StreamWindowTypeInfo<T>(getType()), bufferOperator)
-				.setParallelism(parallelism), keyByKey, transformation, false);
-
-	}
-
-	/**
-	 * Returns the parallelism for the stream discretizer. The returned
-	 * parallelism is either 1 for for non-parallel global policies (or when the
-	 * input stream is non-parallel), environment parallelism for the policies
-	 * that can run in parallel (such as, any ditributed policy, reduce by count
-	 * or time).
-	 * 
-	 * @param transformation
-	 *            The applied transformation
-	 * @return The parallelism for the stream discretizer
-	 */
-	private int getDiscretizerParallelism(WindowTransformation transformation) {
-		return isLocal
-				|| (transformation == WindowTransformation.REDUCEWINDOW && WindowUtils
-						.isParallelPolicy(getTrigger(), getEviction(), dataStream.getParallelism()))
-				|| (discretizerKey != null) ? dataStream.environment.getParallelism() : 1;
-
-	}
-
-	/**
-	 * Dedicated method for applying parallel time reduce transformations on
-	 * windows
-	 * 
-	 * @param reduceFunction
-	 *            Reduce function to apply
-	 * @return The transformed stream
-	 */
-	protected DiscretizedStream<T> timeReduce(ReduceFunction<T> reduceFunction) {
-
-		WindowTransformation transformation = WindowTransformation.REDUCEWINDOW
-				.with(clean(reduceFunction));
-
-		// We get the windowbuffer and set it to emit empty windows with
-		// sequential IDs. This logic is necessary to merge windows created in
-		// parallel.
-		WindowBuffer<T> windowBuffer = getWindowBuffer(transformation).emitEmpty().sequentialID();
-
-		// If there is a groupby for the reduce operation we apply it before the
-		// discretizers, because we will forward everything afterwards to
-		// exploit task chaining
-		if (keyByKey != null) {
-			dataStream = dataStream.keyBy(keyByKey);
-		}
-
-		// We discretize the stream and call the timeReduce function of the
-		// discretized stream, we also pass the type of the windowbuffer
-		DiscretizedStream<T> discretized = discretize(transformation, windowBuffer);
-
-		if (getEviction() instanceof KeepAllEvictionPolicy
-				&& !(windowBuffer instanceof PreAggregator)) {
-			throw new RuntimeException(
-					"Error in preaggregator logic, parallel time reduce should always be preaggregated");
-		}
-
-		return discretized.timeReduce(reduceFunction);
-
-	}
-
-	/**
-	 * Based on the defined policies, returns the stream discretizer to be used
-	 */
-	private OneInputStreamOperator<T, WindowEvent<T>> getDiscretizer() {
-		if (discretizerKey == null) {
-			return new StreamDiscretizer<T>(getTrigger(), getEviction());
-		} else if (getTrigger() instanceof CentralActiveTrigger) {
-			return new GroupedActiveDiscretizer<T>(discretizerKey,
-					(CentralActiveTrigger<T>) getTrigger(),
-					(CloneableEvictionPolicy<T>) getEviction());
-		} else {
-			return new GroupedStreamDiscretizer<T>(discretizerKey,
-					(CloneableTriggerPolicy<T>) getTrigger(),
-					(CloneableEvictionPolicy<T>) getEviction());
-		}
-
-	}
-
-	private OneInputStreamOperator<WindowEvent<T>, StreamWindow<T>> getBufferOperator(
-			WindowBuffer<T> windowBuffer) {
-		if (discretizerKey == null) {
-			return new StreamWindowBuffer<T>(windowBuffer);
-		} else {
-			return new GroupedWindowBuffer<T>(windowBuffer, discretizerKey);
-		}
-	}
-
-	/**
-	 * Based on the given policies returns the WindowBuffer used to store the
-	 * elements in the window. This is the module that also encapsulates the
-	 * pre-aggregator logic when it is applicable, reducing the space cost, and
-	 * trigger latency.
-	 * 
-	 */
-	@SuppressWarnings("unchecked")
-	private WindowBuffer<T> getWindowBuffer(WindowTransformation transformation) {
-		TriggerPolicy<T> trigger = getTrigger();
-		EvictionPolicy<T> eviction = getEviction();
-
-		if (transformation == WindowTransformation.REDUCEWINDOW) {
-			if (WindowUtils.isTumblingPolicy(trigger, eviction)) {
-				if (eviction instanceof KeepAllEvictionPolicy) {
-					if (keyByKey == null) {
-						return new TumblingPreReducer<T>(
-								(ReduceFunction<T>) transformation.getUDF(), getType()
-										.createSerializer(getExecutionConfig())).noEvict();
-					} else {
-						return new TumblingGroupedPreReducer<T>(
-								(ReduceFunction<T>) transformation.getUDF(), keyByKey,
-								getType().createSerializer(getExecutionConfig())).noEvict();
-					}
-				} else {
-					if (keyByKey == null) {
-						return new TumblingPreReducer<T>(
-								(ReduceFunction<T>) transformation.getUDF(), getType()
-										.createSerializer(getExecutionConfig()));
-					} else {
-						return new TumblingGroupedPreReducer<T>(
-								(ReduceFunction<T>) transformation.getUDF(), keyByKey,
-								getType().createSerializer(getExecutionConfig()));
-					}
-				}
-			} else if (WindowUtils.isSlidingCountPolicy(trigger, eviction)) {
-				if (keyByKey == null) {
-					return new SlidingCountPreReducer<T>(
-							clean((ReduceFunction<T>) transformation.getUDF()), dataStream
-									.getType().createSerializer(getExecutionConfig()),
-							WindowUtils.getWindowSize(eviction), WindowUtils.getSlideSize(trigger),
-							((CountTriggerPolicy<?>) trigger).getStart());
-				} else {
-					return new SlidingCountGroupedPreReducer<T>(
-							clean((ReduceFunction<T>) transformation.getUDF()), dataStream
-									.getType().createSerializer(getExecutionConfig()), keyByKey,
-							WindowUtils.getWindowSize(eviction), WindowUtils.getSlideSize(trigger),
-							((CountTriggerPolicy<?>) trigger).getStart());
-				}
-
-			} else if (WindowUtils.isSlidingTimePolicy(trigger, eviction)) {
-				if (keyByKey == null) {
-					return new SlidingTimePreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), dataStream.getType()
-									.createSerializer(getExecutionConfig()),
-							WindowUtils.getWindowSize(eviction), WindowUtils.getSlideSize(trigger),
-							WindowUtils.getTimeStampWrapper(trigger));
-				} else {
-					return new SlidingTimeGroupedPreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), dataStream.getType()
-									.createSerializer(getExecutionConfig()), keyByKey,
-							WindowUtils.getWindowSize(eviction), WindowUtils.getSlideSize(trigger),
-							WindowUtils.getTimeStampWrapper(trigger));
-				}
-
-			} else if (WindowUtils.isJumpingCountPolicy(trigger, eviction)) {
-				if (keyByKey == null) {
-					return new JumpingCountPreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), getType()
-									.createSerializer(getExecutionConfig()),
-							WindowUtils.getSlideSize(trigger) - WindowUtils.getWindowSize(eviction));
-				} else {
-					return new JumpingCountGroupedPreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), keyByKey, getType()
-									.createSerializer(getExecutionConfig()),
-							WindowUtils.getSlideSize(trigger) - WindowUtils.getWindowSize(eviction));
-				}
-			} else if (WindowUtils.isJumpingTimePolicy(trigger, eviction)) {
-				if (keyByKey == null) {
-					return new JumpingTimePreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), getType()
-									.createSerializer(getExecutionConfig()),
-							WindowUtils.getSlideSize(trigger), WindowUtils.getWindowSize(eviction),
-							WindowUtils.getTimeStampWrapper(trigger));
-				} else {
-					return new JumpingTimeGroupedPreReducer<T>(
-							(ReduceFunction<T>) transformation.getUDF(), keyByKey, getType()
-									.createSerializer(getExecutionConfig()),
-							WindowUtils.getSlideSize(trigger), WindowUtils.getWindowSize(eviction),
-							WindowUtils.getTimeStampWrapper(trigger));
-				}
-			}
-		}
-
-		if (eviction instanceof KeepAllEvictionPolicy) {
-			throw new RuntimeException(
-					"Full stream policy can only be used with operations that support preaggregations, such as reduce or aggregations");
-		} else {
-			return new BasicWindowBuffer<T>();
-		}
-	}
-
-	/**
-	 * Applies an aggregation that sums every window of the data stream at the
-	 * given position.
-	 * 
-	 * @param positionToSum
-	 *            The position in the tuple/array to sum
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> sum(int positionToSum) {
-		return aggregate(new SumAggregator<T>(positionToSum, getType(), getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that sums every window of the pojo data stream at
-	 * the given field for every window. </br></br> A field expression is either
-	 * the name of a public field or a getter method with parentheses of the
-	 * stream's underlying type. A dot can be used to drill down into objects,
-	 * as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field to sum
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> sum(String field) {
-		return aggregate(new SumAggregator<T>(field, getType(), getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum value of every window
-	 * of the data stream at the given position.
-	 * 
-	 * @param positionToMin
-	 *            The position to minimize
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> min(int positionToMin) {
-		return aggregate(new ComparableAggregator<T>(positionToMin, getType(), AggregationType.MIN,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum value of the pojo data
-	 * stream at the given field expression for every window. </br></br>A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> min(String field) {
-		return aggregate(new ComparableAggregator<T>(field, getType(), AggregationType.MIN,
-				false, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns the first element by default.
-	 * 
-	 * @param positionToMinBy
-	 *            The position to minimize by
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> minBy(int positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns the first element by default.
-	 * 
-	 * @param positionToMinBy
-	 *            The position to minimize by
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> minBy(String positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns either the first or last one depending
-	 * on the parameter setting.
-	 * 
-	 * @param positionToMinBy
-	 *            The position to minimize
-	 * @param first
-	 *            If true, then the operator return the first element with the
-	 *            minimum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> minBy(int positionToMinBy, boolean first) {
-		return aggregate(new ComparableAggregator<T>(positionToMinBy, getType(), AggregationType.MINBY, first,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum element of the pojo
-	 * data stream by the given field expression for every window. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @param first
-	 *            If True then in case of field equality the first object will
-	 *            be returned
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> minBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<T>(field, getType(), AggregationType.MINBY,
-				first, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum value of every window of
-	 * the data stream at the given position.
-	 * 
-	 * @param positionToMax
-	 *            The position to maximize
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> max(int positionToMax) {
-		return aggregate(new ComparableAggregator<T>(positionToMax, getType(), AggregationType.MAX,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the maximum value of the pojo data
-	 * stream at the given field expression for every window. A field expression
-	 * is either the name of a public field or a getter method with parentheses
-	 * of the {@link DataStream}S underlying type. A dot can be used to drill
-	 * down into objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> max(String field) {
-		return aggregate(new ComparableAggregator<T>(field, getType(), AggregationType.MAX,
-				false, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns the first by default.
-	 * 
-	 * @param positionToMaxBy
-	 *            The position to maximize by
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> maxBy(int positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns the first by default.
-	 * 
-	 * @param positionToMaxBy
-	 *            The position to maximize by
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> maxBy(String positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns either the first or last one depending
-	 * on the parameter setting.
-	 * 
-	 * @param positionToMaxBy
-	 *            The position to maximize by
-	 * @param first
-	 *            If true, then the operator return the first element with the
-	 *            maximum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> maxBy(int positionToMaxBy, boolean first) {
-		return aggregate(new ComparableAggregator<T>(positionToMaxBy, getType(), AggregationType.MAXBY, first,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the maximum element of the pojo
-	 * data stream by the given field expression for every window. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 * 
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @param first
-	 *            If True then in case of field equality the first object will
-	 *            be returned
-	 * @return The transformed DataStream.
-	 */
-	public WindowedDataStream<T> maxBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<T>(field, getType(), AggregationType.MAXBY, first,
-				getExecutionConfig()));
-	}
-
-	private WindowedDataStream<T> aggregate(AggregationFunction<T> aggregator) {
-		return reduceWindow(aggregator);
-	}
-
-	protected TriggerPolicy<T> getTrigger() {
-
-		if (triggerHelper != null) {
-			return triggerHelper.toTrigger();
-		} else if (userTrigger != null) {
-			return userTrigger;
-		} else {
-			throw new RuntimeException("Trigger must not be null");
-		}
-
-	}
-
-	protected EvictionPolicy<T> getEviction() {
-
-		if (evictionHelper != null) {
-			return evictionHelper.toEvict();
-		} else if (userEvicter == null || userEvicter instanceof TumblingEvictionPolicy) {
-			if (triggerHelper instanceof Time) {
-				return triggerHelper.toEvict();
-			} else {
-				return new TumblingEvictionPolicy<T>();
-			}
-		} else {
-			return userEvicter;
-		}
-
-	}
-
-	public <F> F clean(F f) {
-		if (getExecutionConfig().isClosureCleanerEnabled()) {
-			ClosureCleaner.clean(f, true);
-		}
-		ClosureCleaner.ensureSerializable(f);
-		return f;
-	}
-
-	protected boolean isGrouped() {
-		return keyByKey != null;
-	}
-
-	/**
-	 * Gets the output type.
-	 * 
-	 * @return The output type.
-	 */
-	public TypeInformation<T> getType() {
-		return dataStream.getType();
-	}
-
-	public ExecutionConfig getExecutionConfig() {
-		return dataStream.getExecutionConfig();
-	}
-
-	protected WindowedDataStream<T> copy() {
-		return new WindowedDataStream<T>(this);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/RichWindowMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/RichWindowMapFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/RichWindowMapFunction.java
deleted file mode 100644
index ff045a5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/RichWindowMapFunction.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.streaming.api.datastream.WindowedDataStream;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.util.Collector;
-
-/**
- * Abstract class for defining rich mapWindow transformation to be applied on
- * {@link WindowedDataStream}s. The mapWindow function will be called on each
- * {@link StreamWindow}.</p> In addition the user can access the functionality
- * provided by the {@link RichFunction} interface.
- */
-public abstract class RichWindowMapFunction<IN, OUT> extends AbstractRichFunction implements
-		WindowMapFunction<IN, OUT> {
-
-	private static final long serialVersionUID = 9052714915997374185L;
-
-	@Override
-	public abstract void mapWindow(Iterable<IN> values, Collector<OUT> out) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/WindowMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/WindowMapFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/WindowMapFunction.java
deleted file mode 100644
index ececb29..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/WindowMapFunction.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.streaming.api.datastream.WindowedDataStream;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.util.Collector;
-
-/**
- * Interface for defining mapWindow transformation to be applied on
- * {@link WindowedDataStream}s. The mapWindow function will be called on each
- * {@link StreamWindow}.
- */
-public interface WindowMapFunction<T, O> extends Function, Serializable {
-
-	void mapWindow(Iterable<T> values, Collector<O> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java
index 7859b2c..86a12e2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.api.functions.windowing.delta;
 
-import org.apache.flink.streaming.api.windowing.extractor.Extractor;
+import org.apache.flink.streaming.api.functions.windowing.delta.extractor.Extractor;
 
 /**
  * This delta function calculates the cosine distance between two given vectors.

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java
index f9e8ec7..23efbf2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.api.functions.windowing.delta;
 
-import org.apache.flink.streaming.api.windowing.extractor.Extractor;
+import org.apache.flink.streaming.api.functions.windowing.delta.extractor.Extractor;
 
 /**
  * This delta function calculates the euclidean distance between two given

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java
index bd5b0b9..7a4e01a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.api.functions.windowing.delta;
 
-import org.apache.flink.streaming.api.windowing.extractor.Extractor;
+import org.apache.flink.streaming.api.functions.windowing.delta.extractor.Extractor;
 
 /**
  * Extend this abstract class to implement a delta function which is aware of

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTuple.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTuple.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTuple.java
new file mode 100644
index 0000000..baceba4
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ArrayFromTuple.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Converts a Tuple to an Object-Array. The field which should be included in
+ * the array can selected and reordered as needed.
+ */
+public class ArrayFromTuple implements Extractor<Tuple, Object[]> {
+
+	/**
+	 * Auto generated version id
+	 */
+	private static final long serialVersionUID = -6076121226427616818L;
+	int[] order = null;
+
+	/**
+	 * Using this constructor the extractor will convert the whole tuple (all
+	 * fields in the original order) to an array.
+	 */
+	public ArrayFromTuple() {
+		// noting to do
+	}
+
+	/**
+	 * Using this constructor the extractor will combine the fields as specified
+	 * in the indexes parameter in an object array.
+	 * 
+	 * @param indexes
+	 *            the field ids (enumerated from 0)
+	 */
+	public ArrayFromTuple(int... indexes) {
+		this.order = indexes;
+	}
+
+	@Override
+	public Object[] extract(Tuple in) {
+		Object[] output;
+
+		if (order == null) {
+			// copy the whole tuple
+			output = new Object[in.getArity()];
+			for (int i = 0; i < in.getArity(); i++) {
+				output[i] = in.getField(i);
+			}
+		} else {
+			// copy user specified order
+			output = new Object[order.length];
+			for (int i = 0; i < order.length; i++) {
+				output[i] = in.getField(order[i]);
+			}
+		}
+
+		return output;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtract.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtract.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtract.java
new file mode 100644
index 0000000..89c3a32
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/ConcatenatedExtract.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+/**
+ * Combines two extractors which will be executed one after each other.
+ *
+ * @param <FROM>
+ *            The input type of the first extractor.
+ * @param <OVER>
+ *            The output type of the first and the input type of the second
+ *            extractor.
+ * @param <TO>
+ *            The output type of the second extractor and the output type of the
+ *            over all extraction.
+ */
+public class ConcatenatedExtract<FROM, OVER, TO> implements Extractor<FROM, TO> {
+
+	/**
+	 * auto-generated id
+	 */
+	private static final long serialVersionUID = -7807197760725651752L;
+
+	private Extractor<FROM, OVER> e1;
+	private Extractor<OVER, TO> e2;
+
+	/**
+	 * Combines two extractors which will be executed one after each other.
+	 * 
+	 * @param e1
+	 *            First extractor: This extractor gets applied to the input data
+	 *            first. Its output as then passed as input to the second
+	 *            extractor.
+	 * @param e2
+	 *            Second extractor: This extractor gets the output of the first
+	 *            extractor as input. Its output is then the result of the over
+	 *            all extraction.
+	 */
+	public ConcatenatedExtract(Extractor<FROM, OVER> e1, Extractor<OVER, TO> e2) {
+		this.e1 = e1;
+		this.e2 = e2;
+	}
+
+	@Override
+	public TO extract(FROM in) {
+		return e2.extract(e1.extract(in));
+	}
+
+	public <OUT> ConcatenatedExtract<FROM, TO, OUT> add(Extractor<TO, OUT> e3) {
+		return new ConcatenatedExtract<FROM, TO, OUT>(this, e3);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/Extractor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/Extractor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/Extractor.java
new file mode 100644
index 0000000..8cd0014
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/Extractor.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+import java.io.Serializable;
+
+/**
+ * Extractors allow to extract/convert one type to another. They are mostly used
+ * to extract some fields out of a more complex structure (Tuple/Array) to run
+ * further calculation on the extraction result.
+ * 
+ * @param <FROM>
+ *            The input data type.
+ * @param <TO>
+ *            The output data type.
+ */
+public interface Extractor<FROM, TO> extends Serializable {
+
+	/**
+	 * Extracts/Converts the given input to an object of the output type
+	 * 
+	 * @param in
+	 *            the input data
+	 * @return the extracted/converted data
+	 */
+	public TO extract(FROM in);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArray.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArray.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArray.java
new file mode 100644
index 0000000..f9d0a2b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromArray.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.flink.streaming.api.functions.windowing.delta.extractor;
+
+import java.lang.reflect.Array;
+
+/**
+ * Extracts a single field out of an array.
+ * 
+ * @param <OUT>
+ *            The type of the extracted field.
+ */
+public class FieldFromArray<OUT> implements Extractor<Object, OUT> {
+
+	/**
+	 * Auto-gernated version id
+	 */
+	private static final long serialVersionUID = -5161386546695574359L;
+	private int fieldId = 0;
+
+	/**
+	 * Extracts the first field (id 0) from the array
+	 */
+	public FieldFromArray() {
+		// noting to do => will use default 0
+	}
+
+	/**
+	 * Extracts the field with the given id from the array.
+	 * 
+	 * @param fieldId
+	 *            The id of the field which will be extracted from the array.
+	 */
+	public FieldFromArray(int fieldId) {
+		this.fieldId = fieldId;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public OUT extract(Object in) {
+		return (OUT) Array.get(in, fieldId);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTuple.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTuple.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTuple.java
new file mode 100644
index 0000000..627afca
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTuple.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Extracts a single field out of a tuple.
+ * 
+ * @param <OUT>
+ *            The type of the extracted field.
+ */
+public class FieldFromTuple<OUT> implements Extractor<Tuple, OUT> {
+
+	/**
+	 * Auto-gernated version id
+	 */
+	private static final long serialVersionUID = -5161386546695574359L;
+	private int fieldId = 0;
+
+	/**
+	 * Extracts the first field (id 0) from the tuple
+	 */
+	public FieldFromTuple() {
+		// noting to do => will use default 0
+	}
+
+	/**
+	 * Extracts the field with the given id from the tuple.
+	 * 
+	 * @param fieldId
+	 *            The id of the field which will be extracted from the tuple.
+	 */
+	public FieldFromTuple(int fieldId) {
+		this.fieldId = fieldId;
+	}
+
+	@Override
+	public OUT extract(Tuple in) {
+		return in.getField(fieldId);
+	}
+
+}


[08/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArray.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArray.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArray.java
new file mode 100644
index 0000000..b1c080e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArray.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+import java.lang.reflect.Array;
+
+/**
+ * Extracts multiple fields from an array and puts them into a new array of the
+ * specified type.
+ *
+ * @param <OUT>
+ *            The type of the output array. If out is set to String, the output
+ *            of the extractor will be a String[]. If it is set to String[] the
+ *            output will be String[][].
+ */
+public class FieldsFromArray<OUT> implements Extractor<Object, OUT[]> {
+
+	/**
+	 * Auto-generated version id
+	 */
+	private static final long serialVersionUID = 8075055384516397670L;
+	private int[] order;
+	private Class<OUT> clazz;
+
+	/**
+	 * Extracts multiple fields from an array and puts them in the given order
+	 * into a new array of the specified type.
+	 * 
+	 * @param clazz
+	 *            the Class object representing the component type of the new
+	 *            array
+	 * @param indexes
+	 *            The indexes of the fields to be extracted. Any order is
+	 *            possible, but not more than 255 fields due to limitations in
+	 *            {@link Array#newInstance(Class, int...)}.
+	 */
+	public FieldsFromArray(Class<OUT> clazz, int... indexes) {
+		this.order = indexes;
+		this.clazz = clazz;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public OUT[] extract(Object in) {
+		OUT[] output = (OUT[]) Array.newInstance(clazz, order.length);
+		for (int i = 0; i < order.length; i++) {
+			output[i] = (OUT) Array.get(in, this.order[i]);
+		}
+		return output;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.java
new file mode 100644
index 0000000..fc7f3ab
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.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.flink.streaming.api.functions.windowing.delta.extractor;
+
+import org.apache.flink.api.java.tuple.Tuple;
+
+/**
+ * Extracts one or more fields of the type Double from a tuple and puts them
+ * into a new double[]
+ */
+public class FieldsFromTuple implements Extractor<Tuple, double[]> {
+
+	/**
+	 * auto generated version id
+	 */
+	private static final long serialVersionUID = -2554079091050273761L;
+	int[] indexes;
+
+	/**
+	 * Extracts one or more fields of the the type Double from a tuple and puts
+	 * them into a new double[] (in the specified order).
+	 * 
+	 * @param indexes
+	 *            The indexes of the fields to be extracted.
+	 */
+	public FieldsFromTuple(int... indexes) {
+		this.indexes = indexes;
+	}
+
+	@Override
+	public double[] extract(Tuple in) {
+		double[] out = new double[indexes.length];
+		for (int i = 0; i < indexes.length; i++) {
+			out[i] = (Double) in.getField(indexes[i]);
+		}
+		return out;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index cfa6d93..2ca82b1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -575,8 +575,6 @@ public class StreamGraph extends StreamingPlan {
 
 		setJobName(jobGraphName);
 
-		WindowingOptimizer.optimizeGraph(this);
-
 		StreamingJobGraphGenerator jobgraphGenerator = new StreamingJobGraphGenerator(this);
 
 		return jobgraphGenerator.createJobGraph(jobGraphName);
@@ -585,8 +583,6 @@ public class StreamGraph extends StreamingPlan {
 	@Override
 	public String getStreamingPlanAsJSON() {
 
-		WindowingOptimizer.optimizeGraph(this);
-
 		try {
 			return new JSONGenerator(this).getJSON();
 		} catch (JSONException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java
deleted file mode 100644
index cbd2a40..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/WindowingOptimizer.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer;
-import org.apache.flink.streaming.api.operators.windowing.WindowFlattener;
-import org.apache.flink.streaming.api.operators.windowing.WindowMerger;
-
-public class WindowingOptimizer {
-
-	public static void optimizeGraph(StreamGraph streamGraph) {
-
-		// Share common discrtizers
-		setDiscretizerReuse(streamGraph);
-
-		// Remove unnecessary merges before flatten operators
-		removeMergeBeforeFlatten(streamGraph);
-	}
-
-	@SuppressWarnings("rawtypes")
-	private static void removeMergeBeforeFlatten(StreamGraph streamGraph) {
-		Set<Tuple2<Integer, StreamOperator<?>>> operators = streamGraph.getOperators();
-		List<Integer> flatteners = new ArrayList<Integer>();
-
-		for (Tuple2<Integer, StreamOperator<?>> entry : operators) {
-			if (entry.f1 instanceof WindowFlattener) {
-				flatteners.add(entry.f0);
-			}
-		}
-
-		for (Integer flattenerId : flatteners) {
-			// Flatteners should have exactly one input
-			StreamNode input = streamGraph.getStreamNode(flattenerId).getInEdges().get(0)
-					.getSourceVertex();
-
-			// Check whether the flatten is applied after a merge
-			if (input.getOperator() instanceof WindowMerger) {
-
-				// Mergers should have exactly one input
-				StreamNode mergeInput = input.getInEdges().get(0).getSourceVertex();
-
-				// We connect the merge input to the flattener directly
-				streamGraph.addEdge(mergeInput.getId(), flattenerId, 0);
-
-				// If the merger is only connected to the flattener we delete it
-				// completely, otherwise we only remove the edge
-				if (input.getOutEdges().size() > 1) {
-					streamGraph.removeEdge(streamGraph.getStreamEdge(input.getId(), flattenerId));
-				} else {
-					streamGraph.removeVertex(input);
-				}
-
-				streamGraph.setParallelism(flattenerId, mergeInput.getParallelism());
-			}
-		}
-
-	}
-
-	private static void setDiscretizerReuse(StreamGraph streamGraph) {
-
-		Collection<StreamNode> nodes = streamGraph.getStreamNodes();
-		List<StreamNode> discretizers = new ArrayList<StreamNode>();
-
-		for (StreamNode node : nodes) {
-			if (node.getOperator() instanceof StreamDiscretizer) {
-				discretizers.add(node);
-			}
-		}
-
-		List<Tuple2<StreamDiscretizer<?>, List<StreamNode>>> matchingDiscretizers = new ArrayList<Tuple2<StreamDiscretizer<?>, List<StreamNode>>>();
-
-		for (StreamNode discretizer : discretizers) {
-			boolean matchedAny = false;
-			for (Tuple2<StreamDiscretizer<?>, List<StreamNode>> candidate : matchingDiscretizers) {
-
-				Set<Integer> discretizerInEdges = new HashSet<Integer>(
-						discretizer.getInEdgeIndices());
-				Set<Integer> toMatchInEdges = new HashSet<Integer>(candidate.f1.get(0)
-						.getInEdgeIndices());
-
-				boolean partitionersMatch = true;
-
-				for (StreamEdge edge1 : discretizer.getInEdges()) {
-					for (StreamEdge edge2 : candidate.f1.get(0).getInEdges()) {
-						if (edge1.getPartitioner().getClass() != edge2.getPartitioner().getClass()) {
-							partitionersMatch = false;
-						}
-					}
-				}
-
-				if (partitionersMatch
-						&& discretizer.getParallelism() == candidate.f1.get(0).getParallelism()
-						&& discretizer.getOperator().equals(candidate.f0)
-						&& discretizerInEdges.equals(toMatchInEdges)) {
-
-					candidate.f1.add(discretizer);
-					matchedAny = true;
-					break;
-				}
-			}
-			if (!matchedAny) {
-				List<StreamNode> matchingNodes = new ArrayList<StreamNode>();
-				matchingNodes.add(discretizer);
-				matchingDiscretizers.add(new Tuple2<StreamDiscretizer<?>, List<StreamNode>>(
-						(StreamDiscretizer<?>) discretizer.getOperator(), matchingNodes));
-			}
-		}
-
-		for (Tuple2<StreamDiscretizer<?>, List<StreamNode>> matching : matchingDiscretizers) {
-			List<StreamNode> matchList = matching.f1;
-			if (matchList.size() > 1) {
-				StreamNode first = matchList.get(0);
-				for (int i = 1; i < matchList.size(); i++) {
-					replaceDiscretizer(streamGraph, matchList.get(i).getId(), first.getId());
-				}
-			}
-		}
-	}
-
-	private static void replaceDiscretizer(StreamGraph streamGraph, Integer toReplaceID,
-			Integer replaceWithId) {
-		// Convert to array to create a copy
-		List<StreamEdge> outEdges = new ArrayList<StreamEdge>(streamGraph
-				.getStreamNode(toReplaceID).getOutEdges());
-
-		int numOutputs = outEdges.size();
-
-		// Reconnect outputs
-		for (int i = 0; i < numOutputs; i++) {
-			StreamEdge outEdge = outEdges.get(i);
-
-			streamGraph.addEdge(replaceWithId, outEdge.getTargetId(), 0);
-		}
-
-		// Remove the other discretizer
-		streamGraph.removeVertex(streamGraph.getStreamNode(toReplaceID));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/EmptyWindowFilter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/EmptyWindowFilter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/EmptyWindowFilter.java
deleted file mode 100644
index e0fbd89..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/EmptyWindowFilter.java
+++ /dev/null
@@ -1,32 +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.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-
-public class EmptyWindowFilter<OUT> implements FilterFunction<StreamWindow<OUT>> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public boolean filter(StreamWindow<OUT> value) throws Exception {
-		return !value.isEmpty();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java
deleted file mode 100644
index 5141598..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedActiveDiscretizer.java
+++ /dev/null
@@ -1,129 +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.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.policy.CentralActiveTrigger;
-import org.apache.flink.streaming.api.windowing.policy.CloneableEvictionPolicy;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class GroupedActiveDiscretizer<IN> extends GroupedStreamDiscretizer<IN> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(GroupedActiveDiscretizer.class);
-
-	private volatile IN last;
-	private Thread centralThread;
-	private CentralCheck centralCheck;
-
-	public GroupedActiveDiscretizer(KeySelector<IN, ?> keySelector,
-			CentralActiveTrigger<IN> triggerPolicy, CloneableEvictionPolicy<IN> evictionPolicy) {
-		super(keySelector, triggerPolicy, evictionPolicy);
-	}
-
-	@Override
-	protected StreamDiscretizer<IN> makeNewGroup(Object key) throws Exception {
-
-		StreamDiscretizer<IN> groupDiscretizer = new StreamDiscretizer<IN>(triggerPolicy.clone(),
-				evictionPolicy.clone());
-
-		groupDiscretizer.setup(this.output, this.runtimeContext);
-		// We omit the groupDiscretizer.open(...) call here to avoid starting
-		// new active threads
-		return groupDiscretizer;
-	}
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-
-//			last = copy(element);
-			last = element.getValue();
-			Object key = keySelector.getKey(element.getValue());
-
-			synchronized (groupedDiscretizers) {
-				StreamDiscretizer<IN> groupDiscretizer = groupedDiscretizers.get(key);
-
-				if (groupDiscretizer == null) {
-					groupDiscretizer = makeNewGroup(key);
-					groupedDiscretizers.put(key, groupDiscretizer);
-				}
-
-				groupDiscretizer.processRealElement(element);
-			}
-	}
-
-	@Override
-	public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
-		super.open(parameters);
-		centralCheck = new CentralCheck();
-		centralThread = new Thread(centralCheck);
-		centralThread.start();
-	}
-
-	@Override
-	public void dispose() {
-		try {
-			centralCheck.running = false;
-			centralThread.interrupt();
-			centralThread.join();
-		} catch (InterruptedException e) {
-			LOG.info("GroupedActiveDiscretizer got interruped while joining with central thread: {}", e);
-		}
-	}
-
-	private class CentralCheck implements Runnable {
-
-		volatile boolean running = true;
-
-		@Override
-		public void run() {
-			while (running) {
-				// wait for the specified granularity
-				try {
-					Thread.sleep(2000);
-				} catch (InterruptedException e) {
-					// ignore it...
-				}
-
-				try {
-					if (last != null) {
-						synchronized (groupedDiscretizers) {
-							for (StreamDiscretizer<IN> group : groupedDiscretizers.values()) {
-
-								CentralActiveTrigger<IN> groupTrigger = (CentralActiveTrigger<IN>) group.triggerPolicy;
-								Object[] fakes = groupTrigger.notifyOnLastGlobalElement(last);
-								if (fakes != null) {
-									for (Object fake : fakes) {
-										group.triggerOnFakeElement(fake);
-									}
-								}
-							}
-						}
-
-					}
-				} catch (Exception e) {
-					throw new RuntimeException(e);
-				}
-
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java
deleted file mode 100644
index e3cab5c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizer.java
+++ /dev/null
@@ -1,132 +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.flink.streaming.api.operators.windowing;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.policy.CloneableEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.CloneableTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * This operator represents the grouped discretization step of a window
- * transformation. The user supplied eviction and trigger policies are applied
- * on a per group basis to create the {@link StreamWindow} that will be further
- * transformed in the next stages. </p> To allow pre-aggregations supply an
- * appropriate {@link WindowBuffer}.
- */
-public class GroupedStreamDiscretizer<IN> extends StreamDiscretizer<IN> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected KeySelector<IN, ?> keySelector;
-	protected Configuration parameters;
-	protected CloneableTriggerPolicy<IN> triggerPolicy;
-	protected CloneableEvictionPolicy<IN> evictionPolicy;
-
-	protected Map<Object, StreamDiscretizer<IN>> groupedDiscretizers;
-
-	public GroupedStreamDiscretizer(KeySelector<IN, ?> keySelector,
-			CloneableTriggerPolicy<IN> triggerPolicy, CloneableEvictionPolicy<IN> evictionPolicy) {
-
-		super(triggerPolicy, evictionPolicy);
-
-		this.keySelector = keySelector;
-
-		this.triggerPolicy = triggerPolicy;
-		this.evictionPolicy = evictionPolicy;
-
-		this.groupedDiscretizers = new HashMap<Object, StreamDiscretizer<IN>>();
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		for (StreamDiscretizer<IN> group : groupedDiscretizers.values()) {
-			group.emitWindow();
-		}
-	}
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-
-
-			Object key = keySelector.getKey(element.getValue());
-
-			StreamDiscretizer<IN> groupDiscretizer = groupedDiscretizers.get(key);
-
-			if (groupDiscretizer == null) {
-				groupDiscretizer = makeNewGroup(key);
-				groupedDiscretizers.put(key, groupDiscretizer);
-			}
-
-			groupDiscretizer.processRealElement(element);
-
-	}
-
-	/**
-	 * This method creates a new group. The method gets called in case an
-	 * element arrives which has a key which was not seen before. The method
-	 * created a nested {@link StreamDiscretizer} and therefore created clones
-	 * of all distributed trigger and eviction policies.
-	 * 
-	 * @param key
-	 *            The key of the new group.
-	 */
-	protected StreamDiscretizer<IN> makeNewGroup(Object key) throws Exception {
-
-		StreamDiscretizer<IN> groupDiscretizer = new StreamDiscretizer<IN>(triggerPolicy.clone(),
-				evictionPolicy.clone());
-
-		// TODO: this seems very hacky, maybe we can get around this
-		groupDiscretizer.setup(this.output, this.runtimeContext);
-		groupDiscretizer.open(this.parameters);
-
-		return groupDiscretizer;
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof GroupedStreamDiscretizer)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				GroupedStreamDiscretizer<IN> otherDiscretizer = (GroupedStreamDiscretizer<IN>) other;
-
-				return triggerPolicy.equals(otherDiscretizer.triggerPolicy)
-						&& evictionPolicy.equals(otherDiscretizer.evictionPolicy)
-						&& keySelector.equals(otherDiscretizer.keySelector);
-
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "GroupedDiscretizer(Key: " + keySelector.getClass().getSimpleName() + ", Trigger: "
-				+ triggerPolicy.toString() + ", Eviction: " + evictionPolicy.toString() + ")";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java
deleted file mode 100644
index c74b96e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/GroupedWindowBuffer.java
+++ /dev/null
@@ -1,69 +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.flink.streaming.api.operators.windowing;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.WindowEvent;
-import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * This operator flattens the results of the window transformations by
- * outputing the elements of the {@link StreamWindow} one-by-one
- */
-public class GroupedWindowBuffer<T> extends StreamWindowBuffer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private KeySelector<T, ?> keySelector;
-
-	private transient Map<Object, WindowBuffer<T>> windowMap;
-
-	public GroupedWindowBuffer(WindowBuffer<T> buffer, KeySelector<T, ?> keySelector) {
-		super(buffer);
-		this.keySelector = keySelector;
-		this.windowMap = new HashMap<Object, WindowBuffer<T>>();
-	}
-
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-		this.windowMap = new HashMap<Object, WindowBuffer<T>>();
-	}
-
-	@Override
-	public void processElement(StreamRecord<WindowEvent<T>> event) throws Exception {
-		if (event.getValue().getElement() != null) {
-			Object key = keySelector.getKey(event.getValue().getElement());
-			WindowBuffer<T> currentWindow = windowMap.get(key);
-
-			if (currentWindow == null) {
-				currentWindow = buffer.clone();
-				windowMap.put(key, currentWindow);
-			}
-
-			handleWindowEvent(event.getValue(), currentWindow);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelGroupedMerge.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelGroupedMerge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelGroupedMerge.java
deleted file mode 100644
index de947eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelGroupedMerge.java
+++ /dev/null
@@ -1,41 +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.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-
-/**
- * The version of the ParallelMerge CoFlatMap that does not reduce the incoming
- * elements only appends them to the current window. This is necessary for
- * grouped reduces.
- */
-public class ParallelGroupedMerge<OUT> extends ParallelMerge<OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	public ParallelGroupedMerge() {
-		super(null);
-	}
-
-	@Override
-	protected void updateCurrent(StreamWindow<OUT> current, StreamWindow<OUT> nextWindow)
-			throws Exception {
-		current.addAll(nextWindow);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java
deleted file mode 100644
index ce7d887..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMerge.java
+++ /dev/null
@@ -1,145 +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.flink.streaming.api.operators.windowing;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.util.Collector;
-
-/**
- * Class that encapsulates the functionality necessary to merge windows created
- * in parallel. This CoFlatMap uses the information received on the number of
- * parts for each window to merge the different parts. It waits until it
- * receives an indication on the number of parts from all the discretizers
- * before producing any output.
- */
-public class ParallelMerge<OUT> extends
-		RichCoFlatMapFunction<StreamWindow<OUT>, Tuple2<Integer, Integer>, StreamWindow<OUT>> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected Integer numberOfDiscretizers;
-	private ReduceFunction<OUT> reducer;
-
-	private Map<Integer, Integer> availableNumberOfParts = new HashMap<Integer, Integer>();
-	private Map<Integer, Tuple2<StreamWindow<OUT>, Integer>> receivedWindows = new HashMap<Integer, Tuple2<StreamWindow<OUT>, Integer>>();
-	private Map<Integer, Tuple2<Integer, Integer>> receivedNumberOfParts = new HashMap<Integer, Tuple2<Integer, Integer>>();
-
-	public ParallelMerge(ReduceFunction<OUT> reducer) {
-		this.reducer = reducer;
-	}
-
-	@Override
-	public void flatMap1(StreamWindow<OUT> nextWindow, Collector<StreamWindow<OUT>> out)
-			throws Exception {
-
-		Integer id = nextWindow.windowID;
-
-		Tuple2<StreamWindow<OUT>, Integer> current = receivedWindows.get(id);
-
-		if (current == null) {
-			current = new Tuple2<StreamWindow<OUT>, Integer>(nextWindow, 1);
-		} else {
-			updateCurrent(current.f0, nextWindow);
-			current.f1++;
-		}
-
-		Integer count = current.f1;
-
-		if (availableNumberOfParts.containsKey(id) && availableNumberOfParts.get(id) <= count) {
-			out.collect(current.f0);
-			receivedWindows.remove(id);
-			availableNumberOfParts.remove(id);
-
-			checkOld(id);
-
-		} else {
-			receivedWindows.put(id, (Tuple2<StreamWindow<OUT>, Integer>) current);
-		}
-	}
-
-	private void checkOld(Integer id) {
-		// In case we have remaining partial windows (which indicates errors in
-		// processing), output and log them
-		if (receivedWindows.containsKey(id - 1)) {
-			throw new RuntimeException("Error in processing logic, window with id " + id
-					+ " should have already been processed");
-		}
-
-	}
-
-	@Override
-	public void flatMap2(Tuple2<Integer, Integer> partInfo, Collector<StreamWindow<OUT>> out)
-			throws Exception {
-
-		Integer id = partInfo.f0;
-		Integer numOfParts = partInfo.f1;
-
-		Tuple2<Integer, Integer> currentPartInfo = receivedNumberOfParts.get(id);
-		if (currentPartInfo != null) {
-			currentPartInfo.f0 += numOfParts;
-			currentPartInfo.f1++;
-		} else {
-			currentPartInfo = new Tuple2<Integer, Integer>(numOfParts, 1);
-			receivedNumberOfParts.put(id, currentPartInfo);
-		}
-
-		if (currentPartInfo.f1 >= numberOfDiscretizers) {
-			receivedNumberOfParts.remove(id);
-
-			Tuple2<StreamWindow<OUT>, Integer> current = receivedWindows.get(id);
-
-			Integer count = current != null ? current.f1 : -1;
-
-			if (count >= currentPartInfo.f0) {
-				out.collect(current.f0);
-				receivedWindows.remove(id);
-				checkOld(id);
-			} else if (currentPartInfo.f0 > 0) {
-				availableNumberOfParts.put(id, currentPartInfo.f1);
-			}
-		}
-
-	}
-
-	protected void updateCurrent(StreamWindow<OUT> current, StreamWindow<OUT> nextWindow)
-			throws Exception {
-		if (current.size() != 1 || nextWindow.size() != 1) {
-			throw new RuntimeException(
-					"Error in parallel merge logic. Current window should contain only one element.");
-		}
-		OUT currentReduced = current.remove(0);
-		currentReduced = reducer.reduce(currentReduced, nextWindow.get(0));
-		current.add(currentReduced);
-	}
-
-	@Override
-	public void open(Configuration conf) {
-		this.numberOfDiscretizers = getRuntimeContext().getNumberOfParallelSubtasks();
-	}
-
-	Map<Integer, Tuple2<StreamWindow<OUT>, Integer>> getReceivedWindows() {
-		return receivedWindows;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java
deleted file mode 100644
index 74df3ad..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeOperator.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-
-public class ParallelMergeOperator<OUT> extends CoStreamFlatMap<StreamWindow<OUT>, Tuple2<Integer, Integer>, StreamWindow<OUT>> {
-
-	private ParallelMerge<OUT> parallelMerge;
-
-	public ParallelMergeOperator(ParallelMerge<OUT> parallelMerge) {
-		super(parallelMerge);
-		this.parallelMerge = parallelMerge;
-	}
-
-	@Override
-	public void close() throws Exception {
-		// emit remaining (partial) windows
-
-		for (Tuple2<StreamWindow<OUT>, Integer> receivedWindow : parallelMerge.getReceivedWindows().values()) {
-			getCollector().collect(receivedWindow.f0);
-		}
-
-		super.close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java
deleted file mode 100644
index a5e00aa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java
+++ /dev/null
@@ -1,237 +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.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.WindowEvent;
-import org.apache.flink.streaming.api.windowing.policy.ActiveEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.ActiveTriggerCallback;
-import org.apache.flink.streaming.api.windowing.policy.ActiveTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * This operator represents the discretization step of a window transformation.
- * The user supplied eviction and trigger policies are applied to create the
- * {@link StreamWindow} that will be further transformed in the next stages.
- */
-public class StreamDiscretizer<IN>
-		extends AbstractStreamOperator<WindowEvent<IN>>
-		implements OneInputStreamOperator<IN, WindowEvent<IN>> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected TriggerPolicy<IN> triggerPolicy;
-	protected EvictionPolicy<IN> evictionPolicy;
-	private boolean isActiveTrigger;
-	private boolean isActiveEviction;
-	private int bufferSize = 0;
-
-	private transient Thread activePolicyThread;
-
-	protected WindowEvent<IN> windowEvent = new WindowEvent<IN>();
-
-	public StreamDiscretizer(TriggerPolicy<IN> triggerPolicy, EvictionPolicy<IN> evictionPolicy) {
-		this.triggerPolicy = triggerPolicy;
-		this.evictionPolicy = evictionPolicy;
-
-		this.isActiveTrigger = triggerPolicy instanceof ActiveTriggerPolicy;
-		this.isActiveEviction = evictionPolicy instanceof ActiveEvictionPolicy;
-
-		this.chainingStrategy = ChainingStrategy.FORCE_ALWAYS;
-	}
-
-	public TriggerPolicy<IN> getTrigger() {
-		return triggerPolicy;
-	}
-
-	public EvictionPolicy<IN> getEviction() {
-		return evictionPolicy;
-	}
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		processRealElement(element);
-	}
-
-	/**
-	 * This method processed an arrived real element The method is synchronized
-	 * to ensure that it cannot interleave with
-	 * {@link StreamDiscretizer#triggerOnFakeElement(Object)}
-	 * 
-	 * @param input
-	 *            a real input element
-	 * @throws Exception
-	 */
-	protected synchronized void processRealElement(StreamRecord<IN> input) throws Exception {
-
-		// Setting the input element in order to avoid NullFieldException when triggering on fake element
-		windowEvent.setElement(input.getValue());
-		if (isActiveTrigger) {
-			ActiveTriggerPolicy<IN> trigger = (ActiveTriggerPolicy<IN>) triggerPolicy;
-			Object[] result = trigger.preNotifyTrigger(input.getValue());
-			for (Object in : result) {
-				triggerOnFakeElement(in);
-			}
-		}
-
-		boolean isTriggered = false;
-
-		if (triggerPolicy.notifyTrigger(input.getValue())) {
-			emitWindow();
-			isTriggered = true;
-		}
-
-		evict(input.getValue(), isTriggered);
-
-		output.collect(input.replace(windowEvent.setElement(input.getValue())));
-		bufferSize++;
-
-	}
-
-	/**
-	 * This method triggers on an arrived fake element The method is
-	 * synchronized to ensure that it cannot interleave with
-	 * {@link StreamDiscretizer#processRealElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord)}
-	 * 
-	 * @param input
-	 *            a fake input element
-	 */
-	@SuppressWarnings("unchecked")
-	protected synchronized void triggerOnFakeElement(Object input) {
-		if (isActiveEviction) {
-			activeEvict(input);
-			emitWindow();
-		} else {
-			emitWindow();
-			evict((IN) input, true);
-		}
-	}
-
-	/**
-	 * This method emits the content of the buffer as a new {@link StreamWindow}
-	 * if not empty
-	 */
-	protected void emitWindow() {
-		output.collect(new StreamRecord<WindowEvent<IN>>(windowEvent.setTrigger()));
-	}
-
-	private void activeEvict(Object input) {
-		int numToEvict = 0;
-
-		if (isActiveEviction) {
-			ActiveEvictionPolicy<IN> ep = (ActiveEvictionPolicy<IN>) evictionPolicy;
-			numToEvict = ep.notifyEvictionWithFakeElement(input, bufferSize);
-		}
-
-		if (numToEvict > 0) {
-			output.collect(new StreamRecord<WindowEvent<IN>>(windowEvent.setEviction(numToEvict)));
-			bufferSize -= numToEvict;
-			bufferSize = bufferSize >= 0 ? bufferSize : 0;
-		}
-	}
-
-	private void evict(IN input, boolean isTriggered) {
-		int numToEvict = evictionPolicy.notifyEviction(input, isTriggered, bufferSize);
-
-		if (numToEvict > 0) {
-			output.collect(new StreamRecord<WindowEvent<IN>>(windowEvent.setEviction(numToEvict)));
-			bufferSize -= numToEvict;
-			bufferSize = bufferSize >= 0 ? bufferSize : 0;
-		}
-	}
-
-	@Override
-	public void open(org.apache.flink.configuration.Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		if (isActiveTrigger) {
-			ActiveTriggerPolicy<IN> tp = (ActiveTriggerPolicy<IN>) triggerPolicy;
-
-			Runnable runnable = tp.createActiveTriggerRunnable(new WindowingCallback());
-			if (runnable != null) {
-				activePolicyThread = new Thread(runnable);
-				activePolicyThread.start();
-			}
-		}
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		if (activePolicyThread != null) {
-			activePolicyThread.interrupt();
-		}
-
-		emitWindow();
-	}
-
-	@Override
-	public void dispose() {
-		if (activePolicyThread != null) {
-			activePolicyThread.interrupt();
-		}
-	}
-
-	/**
-	 * This class allows the active trigger thread to call back and push fake
-	 * elements at any time.
-	 */
-	private class WindowingCallback implements ActiveTriggerCallback {
-
-		@Override
-		public void sendFakeElement(Object datapoint) {
-			triggerOnFakeElement(datapoint);
-		}
-
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof StreamDiscretizer)
-				|| (other instanceof GroupedStreamDiscretizer)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				StreamDiscretizer<IN> otherDiscretizer = (StreamDiscretizer<IN>) other;
-
-				return triggerPolicy.equals(otherDiscretizer.triggerPolicy)
-						&& evictionPolicy.equals(otherDiscretizer.evictionPolicy);
-
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "Discretizer(Trigger: " + triggerPolicy.toString() + ", Eviction: "
-				+ evictionPolicy.toString() + ")";
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java
deleted file mode 100644
index c057f91..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamWindowBuffer.java
+++ /dev/null
@@ -1,69 +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.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.WindowEvent;
-import org.apache.flink.streaming.api.windowing.windowbuffer.WindowBuffer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * This operator manages the window buffers attached to the discretizers.
- */
-public class StreamWindowBuffer<T>
-		extends AbstractStreamOperator<StreamWindow<T>>
-		implements OneInputStreamOperator<WindowEvent<T>, StreamWindow<T>> {
-
-	private static final long serialVersionUID = 1L;
-
-	protected WindowBuffer<T> buffer;
-
-	public StreamWindowBuffer(WindowBuffer<T> buffer) {
-		this.buffer = buffer;
-		setChainingStrategy(ChainingStrategy.FORCE_ALWAYS);
-		disableInputCopy();
-	}
-
-	@Override
-	public void processElement(StreamRecord<WindowEvent<T>> windowEvent) throws Exception {
-		handleWindowEvent(windowEvent.getValue());
-	}
-
-	protected void handleWindowEvent(WindowEvent<T> windowEvent, WindowBuffer<T> buffer)
-			throws Exception {
-		if (windowEvent.isElement()) {
-			buffer.store(windowEvent.getElement());
-		} else if (windowEvent.isEviction()) {
-			buffer.evict(windowEvent.getEviction());
-		} else if (windowEvent.isTrigger()) {
-			buffer.emitWindow(output);
-		}
-	}
-
-	private void handleWindowEvent(WindowEvent<T> windowEvent) throws Exception {
-		handleWindowEvent(windowEvent, buffer);
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java
deleted file mode 100644
index fa7696a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattener.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * This operator flattens the results of the window transformations by
- * outputing the elements of the {@link StreamWindow} one-by-one
- */
-public class WindowFlattener<T> extends AbstractStreamOperator<T>
-		implements OneInputStreamOperator<StreamWindow<T>, T> {
-
-	private static final long serialVersionUID = 1L;
-
-	public WindowFlattener() {
-		chainingStrategy = ChainingStrategy.FORCE_ALWAYS;
-		disableInputCopy();
-	}
-
-	@Override
-	public void processElement(StreamRecord<StreamWindow<T>> window) throws Exception {
-		for (T element : window.getValue()) {
-			output.collect(new StreamRecord<T>(element));
-		}
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFolder.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFolder.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFolder.java
deleted file mode 100644
index cdfc35b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowFolder.java
+++ /dev/null
@@ -1,100 +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.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.WindowedDataStream;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-
-/**
- * This operator is used to apply foldWindow transformations on
- * {@link WindowedDataStream}s.
- */
-public class WindowFolder<IN, OUT> extends StreamMap<StreamWindow<IN>, StreamWindow<OUT>> {
-
-	private static final long serialVersionUID = 1L;
-
-	FoldFunction<IN, OUT> folder;
-
-	public WindowFolder(FoldFunction<IN, OUT> folder, OUT initialValue) {
-		super(new WindowFoldFunction<IN, OUT>(folder, initialValue));
-		this.folder = folder;
-		disableInputCopy();
-	}
-
-	private static class WindowFoldFunction<IN, OUT> extends AbstractRichFunction implements
-			MapFunction<StreamWindow<IN>, StreamWindow<OUT>> {
-
-		private static final long serialVersionUID = 1L;
-		private OUT initialValue;
-		FoldFunction<IN, OUT> folder;
-
-		public WindowFoldFunction(FoldFunction<IN, OUT> folder, OUT initialValue) {
-			this.folder = folder;
-			this.initialValue = initialValue;
-		}
-
-		@Override
-		public StreamWindow<OUT> map(StreamWindow<IN> window) throws Exception {
-			StreamWindow<OUT> outputWindow = new StreamWindow<OUT>(window.windowID);
-			outputWindow.numberOfParts = window.numberOfParts;
-
-			if (!window.isEmpty()) {
-				OUT accumulator = initialValue;
-				for (int i = 0; i < window.size(); i++) {
-					accumulator = folder.fold(accumulator, window.get(i));
-				}
-				outputWindow.add(accumulator);
-			}
-			return outputWindow;
-		}
-
-		// --------------------------------------------------------------------------------------------
-		//  Forwarding calls to the wrapped folder
-		// --------------------------------------------------------------------------------------------
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			FunctionUtils.openFunction(folder, parameters);
-		}
-
-		@Override
-		public void close() throws Exception {
-			FunctionUtils.closeFunction(folder);
-		}
-
-		@Override
-		public void setRuntimeContext(RuntimeContext t) {
-			FunctionUtils.setFunctionRuntimeContext(folder, t);
-		}
-
-		@Override
-		public RuntimeContext getRuntimeContext() {
-			return FunctionUtils.getFunctionRuntimeContext(folder, getRuntimeContext());
-		}
-
-		// streaming does not use iteration runtime context, so that is omitted
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMapper.java
deleted file mode 100644
index ec4309d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMapper.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.WindowedDataStream;
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-
-/**
- * This operator is used to apply mapWindow transformations on
- * {@link WindowedDataStream}s.
- */
-public class WindowMapper<IN, OUT> extends StreamMap<StreamWindow<IN>, StreamWindow<OUT>> {
-
-	private static final long serialVersionUID = 1L;
-
-	WindowMapFunction<IN, OUT> mapper;
-
-	public WindowMapper(WindowMapFunction<IN, OUT> mapper) {
-		super(new WindowMap<IN, OUT>(mapper));
-		this.mapper = mapper;
-		disableInputCopy();
-	}
-
-	private static class WindowMap<T, R> extends AbstractRichFunction
-			implements MapFunction<StreamWindow<T>, StreamWindow<R>> {
-
-		private static final long serialVersionUID = 1L;
-		WindowMapFunction<T, R> mapper;
-
-		public WindowMap(WindowMapFunction<T, R> mapper) {
-			this.mapper = mapper;
-		}
-
-		@Override
-		public StreamWindow<R> map(StreamWindow<T> window) throws Exception {
-			StreamWindow<R> outputWindow = new StreamWindow<R>(window.windowID);
-
-			outputWindow.numberOfParts = window.numberOfParts;
-
-			mapper.mapWindow(window, outputWindow);
-
-			return outputWindow;
-		}
-
-		// --------------------------------------------------------------------------------------------
-		//  Forwarding calls to the wrapped mapper
-		// --------------------------------------------------------------------------------------------
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			FunctionUtils.openFunction(mapper, parameters);
-		}
-
-		@Override
-		public void close() throws Exception {
-			FunctionUtils.closeFunction(mapper);
-		}
-
-		@Override
-		public void setRuntimeContext(RuntimeContext t) {
-			FunctionUtils.setFunctionRuntimeContext(mapper, t);
-		}
-
-		@Override
-		public RuntimeContext getRuntimeContext() {
-			return FunctionUtils.getFunctionRuntimeContext(mapper, getRuntimeContext());
-		}
-
-		// streaming does not use iteration runtime context, so that is omitted
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java
deleted file mode 100644
index 9ed5e82..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowMerger.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * This operator merges together the different partitions of the
- * {@link StreamWindow}s used to merge the results of parallel transformations
- * that belong in the same window.
- */
-public class WindowMerger<T> extends AbstractStreamOperator<StreamWindow<T>>
-		implements OneInputStreamOperator<StreamWindow<T>, StreamWindow<T>> {
-
-	private static final long serialVersionUID = 1L;
-
-	private Map<Integer, StreamWindow<T>> windows;
-
-	public WindowMerger() {
-		this.windows = new HashMap<Integer, StreamWindow<T>>();
-		chainingStrategy = ChainingStrategy.FORCE_ALWAYS;
-		disableInputCopy();
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public void processElement(StreamRecord<StreamWindow<T>> nextWindowRecord) throws Exception {
-		StreamWindow<T> nextWindow = nextWindowRecord.getValue();
-
-		StreamWindow<T> current = windows.get(nextWindow.windowID);
-
-		if (current == null) {
-			current = nextWindow;
-		} else {
-			current = StreamWindow.merge(current, nextWindow);
-		}
-
-		if (current.numberOfParts == 1) {
-			nextWindowRecord.replace(current);
-			output.collect(nextWindowRecord);
-			windows.remove(nextWindow.windowID);
-		} else {
-			windows.put(nextWindow.windowID, current);
-		}
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartExtractor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartExtractor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartExtractor.java
deleted file mode 100644
index 50b4e7d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartExtractor.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.util.Collector;
-
-/**
- * This FlatMapFunction is used to send the number of parts for each window ID
- * (for each parallel discretizer) to the parallel merger that will use is to
- * merge parallel discretized windows
- */
-public class WindowPartExtractor<OUT> implements FlatMapFunction<StreamWindow<OUT>, Tuple2<Integer, Integer>> {
-
-	private static final long serialVersionUID = 1L;
-
-	Integer lastIndex = -1;
-
-	@Override
-	public void flatMap(StreamWindow<OUT> value, Collector<Tuple2<Integer, Integer>> out)
-			throws Exception {
-
-		// We dont emit new values for the same index, this avoids sending the
-		// same information for the same partitioned window multiple times
-		if (value.windowID != lastIndex) {
-			
-			// For empty windows we send 0 since these windows will be filtered
-			// out
-			if (value.isEmpty()) {
-				out.collect(new Tuple2<Integer, Integer>(value.windowID, 0));
-			} else {
-				out.collect(new Tuple2<Integer, Integer>(value.windowID, value.numberOfParts));
-			}
-			lastIndex = value.windowID;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java
deleted file mode 100644
index 9f31fa0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitioner.java
+++ /dev/null
@@ -1,79 +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.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * This operator applies either split or key partitioning depending on the
- * transformation.
- */
-public class WindowPartitioner<T> extends AbstractStreamOperator<StreamWindow<T>>
-		implements OneInputStreamOperator<StreamWindow<T>, StreamWindow<T>> {
-
-	private static final long serialVersionUID = 1L;
-
-	private KeySelector<T, ?> keySelector;
-	private int numberOfSplits;
-
-	public WindowPartitioner(KeySelector<T, ?> keySelector) {
-		this.keySelector = keySelector;
-
-		chainingStrategy = ChainingStrategy.FORCE_ALWAYS;
-		disableInputCopy();
-	}
-
-	public WindowPartitioner(int numberOfSplits) {
-		this.numberOfSplits = numberOfSplits;
-
-		chainingStrategy = ChainingStrategy.ALWAYS;
-	}
-
-	@Override
-	public void processElement(StreamRecord<StreamWindow<T>> currentWindow) throws Exception {
-
-		if (keySelector == null) {
-			if (numberOfSplits <= 1) {
-				output.collect(currentWindow);
-			} else {
-				StreamWindow<T> unpackedWindow = currentWindow.getValue();
-				for (StreamWindow<T> window : StreamWindow.split(unpackedWindow, numberOfSplits)) {
-					currentWindow.replace(window);
-					output.collect(currentWindow);
-				}
-			}
-		} else {
-
-			for (StreamWindow<T> window : StreamWindow
-					.partitionBy(currentWindow.getValue(), keySelector, true)) {
-				output.collect(new StreamRecord<StreamWindow<T>>(window));
-			}
-
-		}
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowReducer.java
deleted file mode 100644
index a43405e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/WindowReducer.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.WindowedDataStream;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-
-/**
- * This operator is used to apply reduceWindow transformations on
- * {@link WindowedDataStream}s.
- */
-public class WindowReducer<IN> extends StreamMap<StreamWindow<IN>, StreamWindow<IN>> {
-
-	private static final long serialVersionUID = 1L;
-
-	ReduceFunction<IN> reducer;
-
-	public WindowReducer(ReduceFunction<IN> reducer) {
-		super(new WindowReduceFunction<IN>(reducer));
-		this.reducer = reducer;
-		disableInputCopy();
-	}
-
-	private static class WindowReduceFunction<T> extends AbstractRichFunction implements
-			MapFunction<StreamWindow<T>, StreamWindow<T>> {
-
-		private static final long serialVersionUID = 1L;
-		ReduceFunction<T> reducer;
-
-		public WindowReduceFunction(ReduceFunction<T> reducer) {
-			this.reducer = reducer;
-		}
-
-		@Override
-		public StreamWindow<T> map(StreamWindow<T> window) throws Exception {
-			StreamWindow<T> outputWindow = new StreamWindow<T>(window.windowID);
-			outputWindow.numberOfParts = window.numberOfParts;
-
-			if (!window.isEmpty()) {
-				T reduced = window.get(0);
-				for (int i = 1; i < window.size(); i++) {
-					reduced = reducer.reduce(reduced, window.get(i));
-				}
-				outputWindow.add(reduced);
-			}
-			return outputWindow;
-		}
-
-		// --------------------------------------------------------------------------------------------
-		//  Forwarding calls to the wrapped reducer
-		// --------------------------------------------------------------------------------------------
-
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			FunctionUtils.openFunction(reducer, parameters);
-		}
-
-		@Override
-		public void close() throws Exception {
-			FunctionUtils.closeFunction(reducer);
-		}
-
-		@Override
-		public void setRuntimeContext(RuntimeContext t) {
-			FunctionUtils.setFunctionRuntimeContext(reducer, t);
-		}
-
-		@Override
-		public RuntimeContext getRuntimeContext() {
-			return FunctionUtils.getFunctionRuntimeContext(reducer, getRuntimeContext());
-		}
-
-		// streaming does not use iteration runtime context, so that is omitted
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java
deleted file mode 100644
index 5a63940..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindow.java
+++ /dev/null
@@ -1,276 +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.flink.streaming.api.windowing;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.datastream.WindowedDataStream;
-import org.apache.flink.util.Collector;
-
-/**
- * Core abstraction for representing windows for {@link WindowedDataStream}s.
- * The user can apply transformations on these windows with the appropriate
- * {@link WindowedDataStream} methods. </p> Each stream window consists of a
- * random ID, a number representing the number of partitions for this specific
- * window (ID) and the elements itself. The ID and number of parts will be used
- * to merge the subwindows after distributed transformations.
- */
-public class StreamWindow<T> extends ArrayList<T> implements Collector<T> {
-
-	private static final long serialVersionUID = -5150196421193988403L;
-	private static Random rnd = new Random();
-
-	public int windowID;
-	public int numberOfParts;
-
-	/**
-	 * Creates a new window with a random id
-	 */
-	public StreamWindow() {
-		this(rnd.nextInt(), 1);
-	}
-
-	/**
-	 * Creates a new window with the specific id
-	 * 
-	 * @param windowID
-	 *            ID of the window
-	 */
-	public StreamWindow(int windowID) {
-		this(windowID, 1);
-	}
-
-	/**
-	 * Creates a new window with the given id and number of parts
-	 * 
-	 * @param windowID
-	 * @param numberOfParts
-	 */
-	public StreamWindow(int windowID, int numberOfParts) {
-		super();
-		this.windowID = windowID;
-		this.numberOfParts = numberOfParts;
-	}
-
-	/**
-	 * Creates a shallow copy of the window
-	 * 
-	 * @param window
-	 *            The window to be copied
-	 */
-	public StreamWindow(StreamWindow<T> window) {
-		this(window.windowID, window.numberOfParts);
-		addAll(window);
-	}
-
-	/**
-	 * Creates a deep copy of the window using the given serializer
-	 * 
-	 * @param window
-	 *            The window to be copied
-	 * @param serializer
-	 *            The serializer used for copying the records.
-	 */
-	public StreamWindow(StreamWindow<T> window, TypeSerializer<T> serializer) {
-		this(window.windowID, window.numberOfParts);
-		for (T element : window) {
-			add(serializer.copy(element));
-		}
-	}
-
-	/**
-	 * Partitions the window using the given keyselector. A subwindow will be
-	 * created for each key.
-	 * 
-	 * @param streamWindow
-	 *            StreamWindow instance to partition
-	 * @param keySelector
-	 *            The keyselector used for extracting keys.
-	 * @param withKey
-	 *            Flag to decide whether the key object should be included in
-	 *            the created window
-	 * @return A list of the subwindows
-	 */
-	public static <X> List<StreamWindow<X>> partitionBy(StreamWindow<X> streamWindow,
-			KeySelector<X, ?> keySelector, boolean withKey) throws Exception {
-		Map<Object, StreamWindow<X>> partitions = new HashMap<Object, StreamWindow<X>>();
-
-		for (X value : streamWindow) {
-			Object key = keySelector.getKey(value);
-			StreamWindow<X> window = partitions.get(key);
-			if (window == null) {
-				window = new StreamWindow<X>(streamWindow.windowID, 0);
-				partitions.put(key, window);
-			}
-			window.add(value);
-		}
-
-		List<StreamWindow<X>> output = new ArrayList<StreamWindow<X>>();
-		int numkeys = partitions.size();
-
-		for (StreamWindow<X> window : partitions.values()) {
-			output.add(window.setNumberOfParts(numkeys));
-		}
-
-		return output;
-	}
-
-	/**
-	 * Splits the window into n equal (if possible) sizes.
-	 * 
-	 * @param window
-	 *            Window to split
-	 * @param n
-	 *            Number of desired partitions
-	 * @return The list of subwindows.
-	 */
-	public static <X> List<StreamWindow<X>> split(StreamWindow<X> window, int n) {
-		int numElements = window.size();
-		if (n == 0) {
-			return new ArrayList<StreamWindow<X>>();
-		}
-		if (n > numElements) {
-			return split(window, numElements);
-		} else {
-			List<StreamWindow<X>> splitsList = new ArrayList<StreamWindow<X>>();
-			int splitSize = numElements / n;
-
-			int index = -1;
-
-			StreamWindow<X> currentSubWindow = new StreamWindow<X>(window.windowID, n);
-			splitsList.add(currentSubWindow);
-
-			for (X element : window) {
-				index++;
-				if (index == splitSize && splitsList.size() < n) {
-					currentSubWindow = new StreamWindow<X>(window.windowID, n);
-					splitsList.add(currentSubWindow);
-					index = 0;
-				}
-				currentSubWindow.add(element);
-			}
-			return splitsList;
-		}
-	}
-
-	public StreamWindow<T> setNumberOfParts(int n) {
-		this.numberOfParts = n;
-		return this;
-	}
-
-	public void setID(int id) {
-		this.windowID = id;
-	}
-
-	/**
-	 * Checks whether this window can be merged with the given one.
-	 * 
-	 * @param otherWindow
-	 *            The window to test
-	 * @return Window compatibility
-	 */
-	public boolean compatibleWith(StreamWindow<T> otherWindow) {
-		return this.windowID == otherWindow.windowID && this.numberOfParts > 1;
-	}
-
-	/**
-	 * Merges compatible windows together.
-	 * 
-	 * @param windows
-	 *            Windows to merge
-	 * @return Merged window
-	 */
-	public static <R> StreamWindow<R> merge(StreamWindow<R>... windows) {
-		StreamWindow<R> window = new StreamWindow<R>(windows[0]);
-		for (int i = 1; i < windows.length; i++) {
-			StreamWindow<R> next = windows[i];
-			if (window.compatibleWith(next)) {
-				window.addAll(next);
-				window.numberOfParts--;
-			} else {
-				throw new RuntimeException("Can only merge compatible windows");
-			}
-		}
-		return window;
-	}
-
-	/**
-	 * Merges compatible windows together.
-	 * 
-	 * @param windows
-	 *            Windows to merge
-	 * @return Merged window
-	 */
-	public static <R> StreamWindow<R> merge(List<StreamWindow<R>> windows) {
-		if (windows.isEmpty()) {
-			throw new RuntimeException("Need at least one window to merge");
-		} else {
-			StreamWindow<R> window = new StreamWindow<R>(windows.get(0));
-			for (int i = 1; i < windows.size(); i++) {
-				StreamWindow<R> next = windows.get(i);
-				if (window.compatibleWith(next)) {
-					window.addAll(next);
-					window.numberOfParts--;
-				} else {
-					throw new RuntimeException("Can only merge compatible windows");
-				}
-			}
-			return window;
-		}
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		return super.equals(o);
-	}
-
-	@Override
-	public void collect(T record) {
-		add(record);
-	}
-
-	@Override
-	public void close() {
-	}
-
-	@Override
-	public String toString() {
-		return super.toString();
-	}
-
-	/**
-	 * Creates a new {@link StreamWindow} with random id from the given elements
-	 * 
-	 * @param elements
-	 *            The elements contained in the resulting window
-	 * @return The window
-	 */
-	public static <R> StreamWindow<R> fromElements(R... elements) {
-		StreamWindow<R> window = new StreamWindow<R>();
-		for (R element : elements) {
-			window.add(element);
-		}
-		return window;
-	}
-}


[10/10] flink git commit: [hotfix] Change result of WindowedStream ops to SingleOutputStreamOperator

Posted by al...@apache.org.
[hotfix] Change result of WindowedStream ops to SingleOutputStreamOperator


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

Branch: refs/heads/master
Commit: b2b2781fd04a14807849fdc41e4b5ecb8ab75f13
Parents: 0ee0c1f
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Thu Oct 8 11:32:18 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Fri Oct 9 11:15:59 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/AllWindowedStream.java       | 46 +++++++++----------
 .../api/datastream/WindowedStream.java          | 48 ++++++++++----------
 2 files changed, 47 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b2b2781f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index c7a70d7..83e7adc 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -126,14 +126,14 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param function The reduce function.
 	 * @return The data stream that is the result of applying the reduce function to the window. 
 	 */
-	public DataStream<T> reduce(ReduceFunction<T> function) {
+	public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> function) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 
 		String callLocation = Utils.getCallLocationName();
 		String udfName = "Reduce at " + callLocation;
 
-		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
+		SingleOutputStreamOperator<T, ?> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
 		if (result != null) {
 			return result;
 		}
@@ -173,7 +173,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param function The fold function.
 	 * @return The data stream that is the result of applying the fold function to the window.
 	 */
-	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function) {
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 
@@ -191,7 +191,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param function The fold function.
 	 * @return The data stream that is the result of applying the fold function to the window.
 	 */
-	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 		return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
@@ -208,7 +208,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param function The window function.
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
-	public <R> DataStream<R> apply(AllWindowFunction<T, R, W> function) {
+	public <R> SingleOutputStreamOperator<R, ?> apply(AllWindowFunction<T, R, W> function) {
 		TypeInformation<T> inType = input.getType();
 		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
 				function, AllWindowFunction.class, true, true, inType, null, false);
@@ -227,14 +227,14 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param function The window function.
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
-	public <R> DataStream<R> apply(AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
+	public <R> SingleOutputStreamOperator<R, ?> apply(AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 
 		String callLocation = Utils.getCallLocationName();
 		String udfName = "MapWindow at " + callLocation;
 
-		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		SingleOutputStreamOperator<R, ?> result = createFastTimeOperatorIfValid(function, resultType, udfName);
 		if (result != null) {
 			return result;
 		}
@@ -274,7 +274,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param positionToSum The position in the tuple/array to sum
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> sum(int positionToSum) {
+	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
 		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
 	}
 
@@ -291,7 +291,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param field The field to sum
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> sum(String field) {
+	public SingleOutputStreamOperator<T, ?> sum(String field) {
 		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
 	}
 
@@ -302,7 +302,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param positionToMin The position to minimize
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> min(int positionToMin) {
+	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
 		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
 	}
 
@@ -319,7 +319,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param field The field expression based on which the aggregation will be applied.
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> min(String field) {
+	public SingleOutputStreamOperator<T, ?> min(String field) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
 	}
 
@@ -332,7 +332,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 *            The position to minimize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(int positionToMinBy) {
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
 		return this.minBy(positionToMinBy, true);
 	}
 
@@ -344,7 +344,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param positionToMinBy The position to minimize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(String positionToMinBy) {
+	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
 		return this.minBy(positionToMinBy, true);
 	}
 
@@ -358,7 +358,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(int positionToMinBy, boolean first) {
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
 		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
 	}
 
@@ -373,7 +373,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param first If True then in case of field equality the first object will be returned
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(String field, boolean first) {
+	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
 	}
 
@@ -384,7 +384,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param positionToMax The position to maximize
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> max(int positionToMax) {
+	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
 		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
 	}
 
@@ -398,7 +398,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param field The field expression based on which the aggregation will be applied.
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> max(String field) {
+	public SingleOutputStreamOperator<T, ?> max(String field) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
 	}
 
@@ -411,7 +411,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 *            The position to maximize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(int positionToMaxBy) {
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
 		return this.maxBy(positionToMaxBy, true);
 	}
 
@@ -424,7 +424,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 *            The position to maximize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(String positionToMaxBy) {
+	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
 		return this.maxBy(positionToMaxBy, true);
 	}
 
@@ -438,7 +438,7 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(int positionToMaxBy, boolean first) {
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
 		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
 	}
 
@@ -453,11 +453,11 @@ public class AllWindowedStream<T, W extends Window> {
 	 * @param first If True then in case of field equality the first object will be returned
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(String field, boolean first) {
+	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
 	}
 
-	private DataStream<T> aggregate(AggregationFunction<T> aggregator) {
+	private SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregator) {
 		return reduce(aggregator);
 	}
 
@@ -466,7 +466,7 @@ public class AllWindowedStream<T, W extends Window> {
 	// ------------------------------------------------------------------------
 
 
-	private <R> DataStream<R> createFastTimeOperatorIfValid(
+	private <R> SingleOutputStreamOperator<R, ?> createFastTimeOperatorIfValid(
 			Function function,
 			TypeInformation<R> resultType,
 			String functionName) {

http://git-wip-us.apache.org/repos/asf/flink/blob/b2b2781f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 42e0bd7..1b511d8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -137,14 +137,14 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param function The reduce function.
 	 * @return The data stream that is the result of applying the reduce function to the window. 
 	 */
-	public DataStream<T> reduce(ReduceFunction<T> function) {
+	public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> function) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 
 		String callLocation = Utils.getCallLocationName();
 		String udfName = "Reduce at " + callLocation;
 
-		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
+		SingleOutputStreamOperator<T, ?> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
 		if (result != null) {
 			return result;
 		}
@@ -187,7 +187,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param function The fold function.
 	 * @return The data stream that is the result of applying the fold function to the window.
 	 */
-	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function) {
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 
@@ -205,7 +205,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param function The fold function.
 	 * @return The data stream that is the result of applying the fold function to the window.
 	 */
-	public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 		return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
@@ -223,7 +223,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param function The window function.
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
-	public <R> DataStream<R> apply(WindowFunction<T, R, K, W> function) {
+	public <R> SingleOutputStreamOperator<R, ?> apply(WindowFunction<T, R, K, W> function) {
 		TypeInformation<T> inType = input.getType();
 		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
 				function, WindowFunction.class, true, true, inType, null, false);
@@ -243,14 +243,14 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param function The window function.
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
-	public <R> DataStream<R> apply(WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) {
+	public <R> SingleOutputStreamOperator<R, ?> apply(WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) {
 		//clean the closure
 		function = input.getExecutionEnvironment().clean(function);
 
 		String callLocation = Utils.getCallLocationName();
 		String udfName = "MapWindow at " + callLocation;
 
-		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		SingleOutputStreamOperator<R, ?> result = createFastTimeOperatorIfValid(function, resultType, udfName);
 		if (result != null) {
 			return result;
 		}
@@ -276,7 +276,7 @@ public class WindowedStream<T, K, W extends Window> {
 					keySel,
 					new HeapWindowBuffer.Factory<T>(),
 					function,
-					trigger).enableSetProcessingTime(setProcessingTime);;
+					trigger).enableSetProcessingTime(setProcessingTime);
 		}
 
 		return input.transform(opName, resultType, operator);
@@ -293,7 +293,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param positionToSum The position in the tuple/array to sum
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> sum(int positionToSum) {
+	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
 		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
 	}
 
@@ -310,7 +310,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param field The field to sum
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> sum(String field) {
+	public SingleOutputStreamOperator<T, ?> sum(String field) {
 		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
 	}
 
@@ -321,7 +321,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param positionToMin The position to minimize
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> min(int positionToMin) {
+	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
 		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
 	}
 
@@ -338,7 +338,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param field The field expression based on which the aggregation will be applied.
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> min(String field) {
+	public SingleOutputStreamOperator<T, ?> min(String field) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
 	}
 
@@ -351,7 +351,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 *            The position to minimize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(int positionToMinBy) {
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
 		return this.minBy(positionToMinBy, true);
 	}
 
@@ -363,7 +363,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param positionToMinBy The position to minimize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(String positionToMinBy) {
+	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
 		return this.minBy(positionToMinBy, true);
 	}
 
@@ -377,7 +377,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(int positionToMinBy, boolean first) {
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
 		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
 	}
 
@@ -392,7 +392,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param first If True then in case of field equality the first object will be returned
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> minBy(String field, boolean first) {
+	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
 	}
 
@@ -403,7 +403,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param positionToMax The position to maximize
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> max(int positionToMax) {
+	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
 		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
 	}
 
@@ -417,7 +417,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param field The field expression based on which the aggregation will be applied.
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> max(String field) {
+	public SingleOutputStreamOperator<T, ?> max(String field) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
 	}
 
@@ -430,7 +430,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 *            The position to maximize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(int positionToMaxBy) {
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
 		return this.maxBy(positionToMaxBy, true);
 	}
 
@@ -443,7 +443,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 *            The position to maximize by
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(String positionToMaxBy) {
+	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
 		return this.maxBy(positionToMaxBy, true);
 	}
 
@@ -457,7 +457,7 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(int positionToMaxBy, boolean first) {
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
 		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
 	}
 
@@ -472,11 +472,11 @@ public class WindowedStream<T, K, W extends Window> {
 	 * @param first If True then in case of field equality the first object will be returned
 	 * @return The transformed DataStream.
 	 */
-	public DataStream<T> maxBy(String field, boolean first) {
+	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
 		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
 	}
 
-	private DataStream<T> aggregate(AggregationFunction<T> aggregator) {
+	private SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregator) {
 		return reduce(aggregator);
 	}
 
@@ -484,7 +484,7 @@ public class WindowedStream<T, K, W extends Window> {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	private <R> DataStream<R> createFastTimeOperatorIfValid(
+	private <R> SingleOutputStreamOperator<R, ?> createFastTimeOperatorIfValid(
 			Function function,
 			TypeInformation<R> resultType,
 			String functionName) {


[06/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableEvictionPolicy.java
deleted file mode 100644
index 6bc5072..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableEvictionPolicy.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-
-/**
- * When used in grouped windowing, eviction policies must
- * provide a clone method. Eviction policies get cloned to provide an own
- * instance for each group and respectively each individual element buffer as
- * groups maintain their own buffers with the elements belonging to the
- * respective group.
- * 
- * This interface extends {@link EvictionPolicy} with such a clone method. It
- * also adds the Java {@link Cloneable} interface as flag.
- * 
- * @param <DATA>
- *            The data type handled by this policy
- */
-public interface CloneableEvictionPolicy<DATA> extends EvictionPolicy<DATA>, Cloneable {
-
-	/**
-	 * This method should return an exact copy of the object it belongs to
-	 * including the current object state.
-	 * 
-	 * @return a copy of this object
-	 */
-	public CloneableEvictionPolicy<DATA> clone();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiEvictionPolicy.java
deleted file mode 100644
index 5adddc4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiEvictionPolicy.java
+++ /dev/null
@@ -1,84 +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.flink.streaming.api.windowing.policy;
-
-import java.util.LinkedList;
-
-/**
- * This policy does the same as {@link MultiEvictionPolicy}. Additionally it is
- * cloneable and only cloneable policies can be passed to the constructor.
- * 
- * @param <DATA>
- *            The type of data handled by this policy
- */
-public class CloneableMultiEvictionPolicy<DATA> extends MultiEvictionPolicy<DATA> implements
-		CloneableEvictionPolicy<DATA> {
-
-	/**
-	 * Default version id.
-	 */
-	private static final long serialVersionUID = 1L;
-
-	private CloneableEvictionPolicy<DATA>[] allPolicies;
-	private EvictionStrategy strategy;
-
-	/**
-	 * This policy does the same as {@link MultiTriggerPolicy}. Additionally it
-	 * is cloneable and only cloneable policies can be passed to the
-	 * constructor.
-	 * 
-	 * When using this constructor the MAX strategy is used by default. You can
-	 * select other strategies using
-	 * {@link CloneableMultiEvictionPolicy#CloneableMultiEvictionPolicy(EvictionStrategy, CloneableEvictionPolicy...)}
-	 * .
-	 * 
-	 * @param evictionPolicies
-	 *            some cloneable policies to be tied together.
-	 */
-	public CloneableMultiEvictionPolicy(CloneableEvictionPolicy<DATA>... evictionPolicies) {
-		this(EvictionStrategy.MAX, evictionPolicies);
-	}
-
-	/**
-	 * This policy does the same as {@link MultiTriggerPolicy}. Additionally it
-	 * is cloneable and only cloneable policies can be passed to the
-	 * constructor.
-	 * 
-	 * @param strategy
-	 *            the strategy to be used. See {@link MultiEvictionPolicy.EvictionStrategy} for a
-	 *            list of possible options.
-	 * @param evictionPolicies
-	 *            some cloneable policies to be tied together.
-	 */
-	public CloneableMultiEvictionPolicy(EvictionStrategy strategy,
-			CloneableEvictionPolicy<DATA>... evictionPolicies) {
-		super(strategy, evictionPolicies);
-		this.allPolicies = evictionPolicies;
-		this.strategy = strategy;
-	}
-
-	@SuppressWarnings("unchecked")
-	public CloneableEvictionPolicy<DATA> clone() {
-		LinkedList<CloneableEvictionPolicy<DATA>> clonedPolicies = new LinkedList<CloneableEvictionPolicy<DATA>>();
-		for (int i = 0; i < allPolicies.length; i++) {
-			clonedPolicies.add(allPolicies[i].clone());
-		}
-		return new CloneableMultiEvictionPolicy<DATA>(strategy,
-				clonedPolicies.toArray(new CloneableEvictionPolicy[allPolicies.length]));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiTriggerPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiTriggerPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiTriggerPolicy.java
deleted file mode 100644
index aaecefb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableMultiTriggerPolicy.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-import java.util.LinkedList;
-
-/**
- * This policy does the same as {@link MultiTriggerPolicy}. Additionally it is
- * cloneable and only cloneable policies can be passed to the constructor.
- * 
- * @param <DATA>
- *            The type of data handled by this policy
- */
-public class CloneableMultiTriggerPolicy<DATA> extends MultiTriggerPolicy<DATA> implements
-		CloneableTriggerPolicy<DATA>, Cloneable {
-
-	/**
-	 * Default version id.
-	 */
-	private static final long serialVersionUID = 1L;
-
-	private CloneableTriggerPolicy<DATA>[] allPolicies;
-
-	/**
-	 * This policy does the same as {@link MultiTriggerPolicy}. Additionally it
-	 * is cloneable and only cloneable policies can be passed to the
-	 * constructor.
-	 * 
-	 * @param policies
-	 *            some cloneable policies to be tied together.
-	 */
-	public CloneableMultiTriggerPolicy(CloneableTriggerPolicy<DATA>... policies) {
-		super(policies);
-		this.allPolicies = policies;
-	}
-
-	@SuppressWarnings("unchecked")
-	public CloneableTriggerPolicy<DATA> clone() {
-		LinkedList<CloneableTriggerPolicy<DATA>> clonedPolicies = new LinkedList<CloneableTriggerPolicy<DATA>>();
-		for (int i = 0; i < allPolicies.length; i++) {
-			clonedPolicies.add(allPolicies[i].clone());
-		}
-		return new CloneableMultiTriggerPolicy<DATA>(
-				clonedPolicies.toArray(new CloneableTriggerPolicy[allPolicies.length]));
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableTriggerPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableTriggerPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableTriggerPolicy.java
deleted file mode 100644
index 3f55f41..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CloneableTriggerPolicy.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-
-/**
- * When used in grouped windowing, trigger policies can provide
- * a clone method. Cloneable triggers can be used in a distributed manner,
- * which means they get cloned to provide an own instance for each group. This
- * allows each group to trigger individually and only based on the elements
- * belonging to the respective group.
- * 
- * This interface extends {@link TriggerPolicy} with such a clone method. It
- * also adds the Java {@link Cloneable} interface as flag.
- * 
- * @param <DATA>
- *            The data type handled by this policy
- */
-public interface CloneableTriggerPolicy<DATA> extends TriggerPolicy<DATA>, Cloneable {
-
-	/**
-	 * This method should return an exact copy of the object it belongs to
-	 * including the current object state.
-	 * 
-	 * @return a copy of this object
-	 */
-	public CloneableTriggerPolicy<DATA> clone();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicy.java
deleted file mode 100644
index 9be25d3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountEvictionPolicy.java
+++ /dev/null
@@ -1,155 +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.flink.streaming.api.windowing.policy;
-
-/**
- * This eviction policy allows the eviction of data points from the buffer using
- * a counter of arriving elements and a threshold (maximal buffer size)
- * 
- * By default this policy does not react on fake elements. Wrap it in an
- * {@link ActiveEvictionPolicyWrapper} to make it count even fake elements.
- * 
- * @param <IN>
- *            the type of the incoming data points
- */
-public class CountEvictionPolicy<IN> implements CloneableEvictionPolicy<IN> {
-
-	/**
-	 * Auto generated version id
-	 */
-	private static final long serialVersionUID = 2319201348806427996L;
-
-	private int maxElements;
-	private int counter;
-	private int deleteOnEviction = 1;
-	private int startValue;
-
-	/**
-	 * This constructor allows the setup of the simplest possible count based
-	 * eviction. It keeps the size of the buffer according to the given
-	 * maxElements parameter by deleting the oldest element in the buffer.
-	 * Eviction only takes place if the counter of arriving elements would be
-	 * higher than maxElements without eviction.
-	 * 
-	 * @param maxElements
-	 *            The maximum number of elements before eviction. As soon as one
-	 *            more element arrives, the oldest element will be deleted
-	 */
-	public CountEvictionPolicy(int maxElements) {
-		this(maxElements, 1);
-	}
-
-	/**
-	 * This constructor allows to set up both, the maximum number of elements
-	 * and the number of elements to be deleted in case of an eviction.
-	 * 
-	 * Eviction only takes place if the counter of arriving elements would be
-	 * higher than maxElements without eviction. In such a case deleteOnEviction
-	 * elements will be removed from the buffer.
-	 * 
-	 * The counter of arriving elements is adjusted respectively, but never set
-	 * below zero:
-	 * counter=(counter-deleteOnEviction<0)?0:counter-deleteOnEviction
-	 * 
-	 * @param maxElements
-	 *            maxElements The maximum number of elements before eviction.
-	 * @param deleteOnEviction
-	 *            The number of elements to be deleted on eviction. The counter
-	 *            will be adjusted respectively but never below zero.
-	 */
-	public CountEvictionPolicy(int maxElements, int deleteOnEviction) {
-		this(maxElements, deleteOnEviction, 0);
-	}
-
-	/**
-	 * The same as {@link CountEvictionPolicy#CountEvictionPolicy(int, int)}.
-	 * Additionally a custom start value for the counter of arriving elements
-	 * can be set. By setting a negative start value the first eviction can be
-	 * delayed.
-	 * 
-	 * @param maxElements
-	 *            maxElements The maximum number of elements before eviction.
-	 * @param deleteOnEviction
-	 *            The number of elements to be deleted on eviction. The counter
-	 *            will be adjusted respectively but never below zero.
-	 * @param startValue
-	 *            A custom start value for the counter of arriving elements.
-	 * @see CountEvictionPolicy#CountEvictionPolicy(int, int)
-	 */
-	public CountEvictionPolicy(int maxElements, int deleteOnEviction, int startValue) {
-		this.counter = startValue;
-		this.deleteOnEviction = deleteOnEviction;
-		this.maxElements = maxElements;
-		this.startValue = startValue;
-	}
-
-	@Override
-	public int notifyEviction(IN datapoint, boolean triggered, int bufferSize) {
-		// The comparison have to be >= and not == to cover case max=0
-		if (counter >= maxElements) {
-			// Adjust the counter according to the current eviction
-			counter = (counter - deleteOnEviction < 0) ? 0 : counter - deleteOnEviction;
-			// The current element will be added after the eviction
-			// Therefore, increase counter in any case
-			counter++;
-			return deleteOnEviction;
-		} else {
-			counter++;
-			return 0;
-		}
-	}
-
-	@Override
-	public CountEvictionPolicy<IN> clone() {
-		return new CountEvictionPolicy<IN>(maxElements, deleteOnEviction, counter);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof CountEvictionPolicy)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				CountEvictionPolicy<IN> otherPolicy = (CountEvictionPolicy<IN>) other;
-				return startValue == otherPolicy.startValue
-						&& deleteOnEviction == otherPolicy.deleteOnEviction
-						&& maxElements == otherPolicy.maxElements;
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-
-	public int getWindowSize() {
-		return maxElements;
-	}
-
-	public int getStart() {
-		return startValue;
-	}
-	
-	public int getDeleteOnEviction(){
-		return deleteOnEviction;
-	}
-
-	@Override
-	public String toString() {
-		return "CountPolicy(" + maxElements + ")";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicy.java
deleted file mode 100644
index 9bd6f82..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CountTriggerPolicy.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-/**
- * This policy triggers at every n'th element.
- * 
- * @param <IN>
- *            The type of the data points which are handled by this policy
- */
-public class CountTriggerPolicy<IN> implements CloneableTriggerPolicy<IN> {
-
-	/**
-	 * Auto generated version ID
-	 */
-	private static final long serialVersionUID = -6357200688886103968L;
-
-	public static final int DEFAULT_START_VALUE = 0;
-
-	private int counter;
-	private int max;
-	private int startValue;
-
-	/**
-	 * This constructor will set up a count based trigger, which triggers after
-	 * max elements have arrived.
-	 * 
-	 * @param max
-	 *            The number of arriving elements before the trigger occurs.
-	 */
-	public CountTriggerPolicy(int max) {
-		this(max, DEFAULT_START_VALUE);
-	}
-
-	/**
-	 * In addition to {@link CountTriggerPolicy#CountTriggerPolicy(int)} this
-	 * constructor allows to set a custom start value for the element counter.
-	 * This can be used to delay the first trigger by setting a negative start
-	 * value. Often the first trigger should be delayed in case of sliding
-	 * windows. For example if the size of a window should be 4 and a trigger
-	 * should happen every 2, a start value of -2 would allow to also have the
-	 * first window of size 4.
-	 * 
-	 * @param max
-	 *            The number of arriving elements before the trigger occurs.
-	 * @param startValue
-	 *            The start value for the counter of arriving elements.
-	 * @see CountTriggerPolicy#CountTriggerPolicy(int)
-	 */
-	public CountTriggerPolicy(int max, int startValue) {
-		this.max = max;
-		this.counter = startValue;
-		this.startValue = startValue;
-	}
-
-	@Override
-	public boolean notifyTrigger(IN datapoint) {
-		// The comparison have to be >= and not == to cover case max=0
-		if (counter >= max) {
-			// The current data point will be part of the next window!
-			// Therefore the counter needs to be set to one already.
-			counter = 1;
-			return true;
-		} else {
-			counter++;
-			return false;
-		}
-	}
-
-	@Override
-	public CountTriggerPolicy<IN> clone() {
-		return new CountTriggerPolicy<IN>(max, counter);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof CountTriggerPolicy)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				CountTriggerPolicy<IN> otherPolicy = (CountTriggerPolicy<IN>) other;
-				return max == otherPolicy.max && startValue == otherPolicy.startValue;
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-
-	public int getSlideSize() {
-		return max;
-	}
-	
-	public int getStart() {
-		return startValue;
-	}
-
-	@Override
-	public String toString() {
-		return "CountPolicy(" + max + ")";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicy.java
deleted file mode 100644
index 0b6a493..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/DeltaPolicy.java
+++ /dev/null
@@ -1,167 +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.flink.streaming.api.windowing.policy;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-import org.apache.flink.runtime.util.DataOutputSerializer;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-
-/**
- * This policy calculates a delta between the data point which triggered last
- * and the currently arrived data point. It triggers if the delta is higher than
- * a specified threshold.
- * 
- * In case it gets used for eviction, this policy starts from the first element
- * of the buffer and removes all elements from the buffer which have a higher
- * delta then the threshold. As soon as there is an element with a lower delta,
- * the eviction stops.
- * 
- * By default this policy does not react on fake elements. Wrap it in an
- * {@link ActiveEvictionPolicyWrapper} to make it calculate the delta even on
- * fake elements.
- * 
- * @param <DATA>
- *            The type of the data points which are handled by this policy
- */
-public class DeltaPolicy<DATA> implements CloneableTriggerPolicy<DATA>,
-		CloneableEvictionPolicy<DATA> {
-
-	/**
-	 * Auto generated version ID
-	 */
-	private static final long serialVersionUID = -7797538922123394967L;
-
-	//Used for serializing the threshold
-	private final static int INITIAL_SERIALIZER_BYTES = 1024;
-
-	protected DeltaFunction<DATA> deltaFuntion;
-	private List<DATA> windowBuffer;
-	protected double threshold;
-	private TypeSerializer<DATA> typeSerializer;
-	protected transient DATA triggerDataPoint;
-
-	/**
-	 * Creates a delta policy which calculates a delta between the data point
-	 * which triggered last and the currently arrived data point. It triggers if
-	 * the delta is higher than a specified threshold. As the data may be sent to
-	 * the cluster a {@link TypeSerializer} is needed for the initial value.
-	 *
-	 * <p>
-	 * In case it gets used for eviction, this policy starts from the first
-	 * element of the buffer and removes all elements from the buffer which have
-	 * a higher delta then the threshold. As soon as there is an element with a
-	 * lower delta, the eviction stops.
-	 * </p>
-	 *
-	 * @param deltaFuntion
-	 * 				The delta function to be used.
-	 * @param init
-	 *				The initial to be used for the calculation of a delta before
-	 *				the first trigger.
-	 * @param threshold
-	 * 				The threshold upon which a triggering should happen.
-	 * @param typeSerializer
-	 * 				TypeSerializer to properly forward the initial value to
-	 * 				the cluster
-	 */
-	@SuppressWarnings("unchecked")
-	public DeltaPolicy(DeltaFunction<DATA> deltaFuntion, DATA init, double threshold, TypeSerializer typeSerializer) {
-		this.deltaFuntion = deltaFuntion;
-		this.triggerDataPoint = init;
-		this.windowBuffer = new LinkedList<DATA>();
-		this.threshold = threshold;
-		this.typeSerializer = typeSerializer;
-	}
-
-	@Override
-	public boolean notifyTrigger(DATA datapoint) {
-		if (deltaFuntion.getDelta(this.triggerDataPoint, datapoint) > this.threshold) {
-			this.triggerDataPoint = datapoint;
-			return true;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public int notifyEviction(DATA datapoint, boolean triggered, int bufferSize) {
-		windowBuffer = windowBuffer.subList(windowBuffer.size() - bufferSize, bufferSize);
-		int evictCount = 0;
-		for (DATA bufferPoint : windowBuffer) {
-			if (deltaFuntion.getDelta(bufferPoint, datapoint) < this.threshold) {
-				break;
-			}
-			evictCount++;
-		}
-
-		if (evictCount > 0) {
-			windowBuffer = windowBuffer.subList(evictCount, windowBuffer.size());
-		}
-		windowBuffer.add(datapoint);
-		return evictCount;
-	}
-
-	@Override
-	public DeltaPolicy<DATA> clone() {
-		return new DeltaPolicy<DATA>(deltaFuntion, triggerDataPoint, threshold, typeSerializer);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof DeltaPolicy)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				DeltaPolicy<DATA> otherPolicy = (DeltaPolicy<DATA>) other;
-				return threshold == otherPolicy.threshold
-						&& deltaFuntion.getClass() == otherPolicy.deltaFuntion.getClass()
-						&& triggerDataPoint.equals(otherPolicy.triggerDataPoint);
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "DeltaPolicy(" + threshold + ", " + deltaFuntion.getClass().getSimpleName() + ")";
-	}
-
-	private void writeObject(ObjectOutputStream stream) throws IOException{
-		stream.defaultWriteObject();
-		DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(INITIAL_SERIALIZER_BYTES);
-		typeSerializer.serialize(triggerDataPoint, dataOutputSerializer);
-		stream.write(dataOutputSerializer.getByteArray());
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException {
-		stream.defaultReadObject();
-		byte[] bytes = new byte[stream.available()];
-		stream.readFully(bytes);
-		triggerDataPoint = typeSerializer.deserialize(new DataInputDeserializer(bytes, 0, bytes.length));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/EvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/EvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/EvictionPolicy.java
deleted file mode 100644
index b95053a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/EvictionPolicy.java
+++ /dev/null
@@ -1,47 +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.flink.streaming.api.windowing.policy;
-
-import java.io.Serializable;
-
-/**
- * An eviction policy specifies under which condition data points should be
- * deleted from the buffer. Deletions must be done only in the order the
- * elements arrived. Therefore, the policy only returns the number of elements
- * to evict on each element arrival.
- * 
- * @param <DATA>
- *            the type of the data handled by this policy
- */
-public interface EvictionPolicy<DATA> extends Serializable {
-
-	/**
-	 * Proves if and how many elements should be deleted from the element
-	 * buffer. The eviction takes place after the trigger and after the call to
-	 * the UDF but before the adding of the new data point.
-	 *
-	 * @param datapoint
-	 *            data point the data point which arrived
-	 * @param triggered
-	 *            Information whether the UDF was triggered or not
-	 * @param bufferSize
-	 *            The current size of the element buffer at the operator
-	 * @return The number of elements to be deleted from the buffer
-	 */
-	public int notifyEviction(DATA datapoint, boolean triggered, int bufferSize);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/KeepAllEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/KeepAllEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/KeepAllEvictionPolicy.java
deleted file mode 100644
index 6fad749..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/KeepAllEvictionPolicy.java
+++ /dev/null
@@ -1,29 +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.flink.streaming.api.windowing.policy;
-
-public class KeepAllEvictionPolicy<T> implements EvictionPolicy<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public int notifyEviction(T datapoint, boolean triggered, int bufferSize) {
-		return 0;
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicy.java
deleted file mode 100644
index 79e8119..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiEvictionPolicy.java
+++ /dev/null
@@ -1,170 +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.flink.streaming.api.windowing.policy;
-
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * This policy provides the ability to use multiple eviction policies at the
- * same time. It allows to use both, active and not active evictions.
- * 
- * @param <DATA>
- *            The type of data-items handled by the policies
- */
-public class MultiEvictionPolicy<DATA> implements ActiveEvictionPolicy<DATA> {
-
-	/**
-	 * Default version id.
-	 */
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * This enum provides the different options for the eviction strategy.
-	 * 
-	 * You can choose from the following:
-	 * <ul>
-	 * <li>MIN: The number of elements to evict will be the smallest one which
-	 * is greater than 0 and was returned by any of the given policies. If all
-	 * policies return 0, the result is 0.</li>
-	 * <li>MAX: The number of elements to evict will be the greatest one which
-	 * was returned by any of the given policies.</li>
-	 * <li>SUM: The number of elements to evict will be the sum of all values
-	 * returned by the nested eviction policies.</li>
-	 * <li>PRIORITY: Depending on the order in which the policies have been
-	 * passed to the constructor, the first return value greater than 0 will be
-	 * the the number of elements to evict. If all policies return 0, the result
-	 * is 0.</li>
-	 * </ul>
-	 */
-	public enum EvictionStrategy {
-		MIN, MAX, SUM, PRIORITY
-	}
-
-	private List<EvictionPolicy<DATA>> allEvictionPolicies;
-	private List<ActiveEvictionPolicy<DATA>> activeEvictionPolicies;
-	private EvictionStrategy selectedStrategy;
-
-	/**
-	 * This policy provides the ability to use multiple eviction policies at the
-	 * same time. It allows to use both, active and not active evictions.
-	 * 
-	 * When using this constructor the MAX strategy is used by default. You can
-	 * select other strategies using
-	 * {@link MultiEvictionPolicy#MultiEvictionPolicy(EvictionStrategy, EvictionPolicy...)}
-	 * .
-	 * 
-	 * @param evictionPolicies
-	 *            Any active or not active eviction policies. Both types can be
-	 *            used at the same time.
-	 */
-	public MultiEvictionPolicy(EvictionPolicy<DATA>... evictionPolicies) {
-		this(EvictionStrategy.MAX, evictionPolicies);
-	}
-
-	/**
-	 * This policy provides the ability to use multiple eviction policies at the
-	 * same time. It allows to use both, active and not active evictions.
-	 * 
-	 * @param strategy
-	 *            the strategy to be used. See {@link EvictionStrategy} for a
-	 *            list of possible options.
-	 * @param evictionPolicies
-	 *            Any active or not active eviction policies. Both types can be
-	 *            used at the same time.
-	 */
-	public MultiEvictionPolicy(EvictionStrategy strategy, EvictionPolicy<DATA>... evictionPolicies) {
-		// initialize lists of policies
-		this.allEvictionPolicies = new LinkedList<EvictionPolicy<DATA>>();
-		this.activeEvictionPolicies = new LinkedList<ActiveEvictionPolicy<DATA>>();
-
-		// iterate over policies and add them to the lists
-		for (EvictionPolicy<DATA> ep : evictionPolicies) {
-			this.allEvictionPolicies.add(ep);
-			if (ep instanceof ActiveEvictionPolicy) {
-				this.activeEvictionPolicies.add((ActiveEvictionPolicy<DATA>) ep);
-			}
-		}
-
-		// Remember eviction strategy
-		this.selectedStrategy = strategy;
-	}
-
-	@Override
-	public int notifyEviction(DATA datapoint, boolean triggered, int bufferSize) {
-		LinkedList<Integer> results = new LinkedList<Integer>();
-		for (EvictionPolicy<DATA> policy : allEvictionPolicies) {
-			results.add(policy.notifyEviction(datapoint, triggered, bufferSize));
-		}
-		return getNumToEvict(results);
-	}
-
-	@Override
-	public int notifyEvictionWithFakeElement(Object datapoint, int bufferSize) {
-		LinkedList<Integer> results = new LinkedList<Integer>();
-		for (ActiveEvictionPolicy<DATA> policy : activeEvictionPolicies) {
-			results.add(policy.notifyEvictionWithFakeElement(datapoint, bufferSize));
-		}
-		return getNumToEvict(results);
-	}
-
-	private int getNumToEvict(LinkedList<Integer> items) {
-		int result;
-		switch (selectedStrategy) {
-
-		case MIN:
-			result = Integer.MAX_VALUE;
-			for (Integer item : items) {
-				if (result > item) {
-					result = item;
-				}
-			}
-			return result;
-
-		case MAX:
-			result = 0;
-			for (Integer item : items) {
-				if (result < item) {
-					result = item;
-				}
-			}
-			return result;
-
-		case SUM:
-			result = 0;
-			for (Integer item : items) {
-				result += item;
-			}
-			return result;
-
-		case PRIORITY:
-			for (Integer item : items) {
-				if (item > 0) {
-					return item;
-				}
-			}
-			return 0;
-
-		default:
-			// The following line should never be reached. Just for the
-			// compiler.
-			return 0;
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicy.java
deleted file mode 100644
index a3c6a22..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/MultiTriggerPolicy.java
+++ /dev/null
@@ -1,123 +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.flink.streaming.api.windowing.policy;
-
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * This class allows to use multiple trigger policies at the same time. It
- * allows to use both, active and not active triggers.
- * 
- * @param <DATA>
- *            the data type handled by this policy
- */
-public class MultiTriggerPolicy<DATA> implements ActiveTriggerPolicy<DATA> {
-
-	/**
-	 * Default version id.
-	 */
-	private static final long serialVersionUID = 1L;
-
-	private List<TriggerPolicy<DATA>> allTriggerPolicies;
-	private List<ActiveTriggerPolicy<DATA>> activeTriggerPolicies;
-
-	/**
-	 * This policy allows to use multiple trigger policies at the same time. It
-	 * allows to use both, active and not active triggers.
-	 * 
-	 * This policy triggers in case at least one of the nested policies
-	 * triggered. If active policies are nested all produces fake elements will
-	 * be returned.
-	 * 
-	 * @param policies
-	 *            Any active or not active trigger policies. Both types can be
-	 *            used at the same time.
-	 */
-	public MultiTriggerPolicy(TriggerPolicy<DATA>... policies) {
-		allTriggerPolicies = new LinkedList<TriggerPolicy<DATA>>();
-		activeTriggerPolicies = new LinkedList<ActiveTriggerPolicy<DATA>>();
-
-		for (TriggerPolicy<DATA> policy : policies) {
-			this.allTriggerPolicies.add(policy);
-			if (policy instanceof ActiveTriggerPolicy) {
-				this.activeTriggerPolicies.add((ActiveTriggerPolicy<DATA>) policy);
-			}
-		}
-	}
-
-	@Override
-	public boolean notifyTrigger(DATA datapoint) {
-		boolean trigger = false;
-		for (TriggerPolicy<DATA> policy : allTriggerPolicies) {
-			if (policy.notifyTrigger(datapoint)) {
-				trigger = true;
-				// Do not at a break here. All trigger must see the element!
-			}
-		}
-		return trigger;
-	}
-
-	@Override
-	public Object[] preNotifyTrigger(DATA datapoint) {
-		List<Object> fakeElements = new LinkedList<Object>();
-		for (ActiveTriggerPolicy<DATA> policy : activeTriggerPolicies) {
-			for (Object fakeElement : policy.preNotifyTrigger(datapoint)) {
-				fakeElements.add(fakeElement);
-			}
-		}
-		return fakeElements.toArray();
-	}
-
-	@Override
-	public Runnable createActiveTriggerRunnable(ActiveTriggerCallback callback) {
-		List<Runnable> runnables = new LinkedList<Runnable>();
-		for (ActiveTriggerPolicy<DATA> policy : activeTriggerPolicies) {
-			Runnable tmp = policy.createActiveTriggerRunnable(callback);
-			if (tmp != null) {
-				runnables.add(tmp);
-			}
-		}
-		if (runnables.size() == 0) {
-			return null;
-		} else {
-			return new MultiActiveTriggerRunnable(runnables);
-		}
-	}
-
-	/**
-	 * This class serves a nest for all active trigger runnables. Once the run
-	 * method gets executed, all the runnables are started in own threads.
-	 */
-	private class MultiActiveTriggerRunnable implements Runnable {
-
-		List<Runnable> runnables;
-
-		MultiActiveTriggerRunnable(List<Runnable> runnables) {
-			this.runnables = runnables;
-		}
-
-		@Override
-		public void run() {
-			for (Runnable runnable : runnables) {
-				new Thread(runnable).start();
-			}
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicy.java
deleted file mode 100644
index eaa8063..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/PunctuationPolicy.java
+++ /dev/null
@@ -1,147 +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.flink.streaming.api.windowing.policy;
-
-import org.apache.flink.streaming.api.windowing.extractor.Extractor;
-
-/**
- * This policy can be used to trigger and evict based on a punctuation which is
- * present within the arriving data. Using this policy, one can react on an
- * externally defined arbitrary windowing semantic.
- * 
- * In case this policy is used for eviction, the complete buffer will get
- * deleted in case the punctuation is detected.
- * 
- * By default this policy does not react on fake elements. Wrap it in an
- * {@link ActiveEvictionPolicyWrapper} to make it react on punctuation even in
- * fake elements.
- * 
- * @param <IN>
- *            The type of the input data handled by this policy. An
- *            {@link Extractor} can be used to extract DATA for IN.
- * @param <DATA>
- *            The type of the punctuation. An {@link Extractor} can be used to
- *            extract DATA for IN.
- */
-public class PunctuationPolicy<IN, DATA> implements CloneableTriggerPolicy<IN>,
-		CloneableEvictionPolicy<IN> {
-
-	/**
-	 * auto generated version id
-	 */
-	private static final long serialVersionUID = -8845130188912602498L;
-	private int counter = 0;
-	private Extractor<IN, DATA> extractor;
-	private DATA punctuation;
-
-	/**
-	 * Creates the punctuation policy without using any extractor. To make this
-	 * work IN and DATA must not be different types.
-	 * 
-	 * @param punctuation
-	 *            the punctuation which leads to trigger/evict.
-	 */
-	public PunctuationPolicy(DATA punctuation) {
-		this(punctuation, null);
-	}
-
-	/**
-	 * Creates the punctuation policy which uses the specified extractor to
-	 * isolate the punctuation from the data.
-	 * 
-	 * @param punctuation
-	 *            the punctuation which leads to trigger/evict.
-	 * @param extractor
-	 *            An {@link Extractor} which converts IN to DATA.
-	 */
-	public PunctuationPolicy(DATA punctuation, Extractor<IN, DATA> extractor) {
-		this.punctuation = punctuation;
-		this.extractor = extractor;
-	}
-
-	@Override
-	public int notifyEviction(IN datapoint, boolean triggered, int bufferSize) {
-		if (notifyTrigger(datapoint)) {
-			int tmp = counter;
-			// As the current will be add after the eviction the counter needs
-			// to be set to one already
-			counter = 1;
-			return tmp;
-		} else {
-			counter++;
-			return 0;
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public boolean notifyTrigger(IN datapoint) {
-		DATA tmp;
-
-		// eventually extract data
-		if (extractor == null) {
-			// unchecked convert (cannot check it here)
-			tmp = (DATA) datapoint;
-		} else {
-			tmp = extractor.extract(datapoint);
-		}
-
-		// compare data with punctuation
-		if (punctuation.equals(tmp)) {
-			return true;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public PunctuationPolicy<IN, DATA> clone() {
-		return new PunctuationPolicy<IN, DATA>(punctuation, extractor);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof PunctuationPolicy)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				PunctuationPolicy<IN, DATA> otherPolicy = (PunctuationPolicy<IN, DATA>) other;
-				if (extractor != null) {
-					return extractor.getClass() == otherPolicy.extractor.getClass()
-							&& punctuation.equals(otherPolicy.punctuation);
-				} else {
-					return punctuation.equals(otherPolicy.punctuation)
-							&& otherPolicy.extractor == null;
-				}
-
-			} catch (Exception e) {
-				return false;
-			}
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "PunctuationPolicy(" + punctuation
-				+ (extractor != null
-					? ", " + extractor.getClass().getSimpleName()
-					: "")
-				+ ")";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicy.java
deleted file mode 100644
index ae17e29..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeEvictionPolicy.java
+++ /dev/null
@@ -1,167 +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.flink.streaming.api.windowing.policy;
-
-import java.util.LinkedList;
-
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-
-/**
- * This eviction policy evicts all elements which are older then a specified
- * time. The time is measured using a given {@link Timestamp} implementation. A
- * point in time is always represented as long. Therefore, the granularity can
- * be set as long value as well.
- * 
- * @param <DATA>
- *            The type of the incoming data points which are processed by this
- *            policy.
- */
-public class TimeEvictionPolicy<DATA> implements ActiveEvictionPolicy<DATA>,
-		CloneableEvictionPolicy<DATA> {
-
-	/**
-	 * auto generated version id
-	 */
-	private static final long serialVersionUID = -1457476766124518220L;
-
-	private long granularity;
-	private TimestampWrapper<DATA> timestampWrapper;
-	private LinkedList<Long> buffer = new LinkedList<Long>();
-
-	/**
-	 * This eviction policy evicts all elements which are older than a specified
-	 * time. The time is measured using a given {@link Timestamp}
-	 * implementation. A point in time is always represented as long. Therefore,
-	 * the granularity can be set as long value as well. If this value is set to
-	 * 2 the policy will evict all elements which are older as 2.
-	 * 
-	 * <code>
-	 *   while (time(firstInBuffer)<current time-granularity){
-	 *   	evict firstInBuffer;
-	 *   }
-	 * </code>
-	 * 
-	 * @param granularity
-	 *            The granularity of the eviction. If this value is set to 2 the
-	 *            policy will evict all elements which are older as 2(if
-	 *            (time(X)<current time-granularity) evict X).
-	 * @param timestampWrapper
-	 *            The {@link TimestampWrapper} to measure the time with. This
-	 *            can be either user defined of provided by the API.
-	 */
-	public TimeEvictionPolicy(long granularity, TimestampWrapper<DATA> timestampWrapper) {
-		this.timestampWrapper = timestampWrapper;
-		this.granularity = granularity;
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public int notifyEvictionWithFakeElement(Object datapoint, int bufferSize) {
-		checkForDeleted(bufferSize);
-
-		long threshold;
-		try {
-			threshold = (Long) datapoint - granularity;
-		} catch (ClassCastException e) {
-			threshold = timestampWrapper.getTimestamp((DATA) datapoint) - granularity;
-		}
-
-		// return result
-		return deleteAndCountExpired(threshold);
-
-	}
-
-	@Override
-	public int notifyEviction(DATA datapoint, boolean triggered, int bufferSize) {
-
-		checkForDeleted(bufferSize);
-
-		// remember timestamp
-		long time = timestampWrapper.getTimestamp(datapoint);
-
-		// delete and count expired tuples
-		long threshold = time - granularity;
-		int counter = deleteAndCountExpired(threshold);
-
-		// Add current element to buffer
-		buffer.add(time);
-
-		// return result
-		return counter;
-
-	}
-
-	private void checkForDeleted(int bufferSize) {
-		// check for deleted tuples (deletes by other policies)
-		while (bufferSize < this.buffer.size()) {
-			this.buffer.removeFirst();
-		}
-	}
-
-	private int deleteAndCountExpired(long threshold) {
-		int counter = 0;
-		while (!buffer.isEmpty()) {
-
-			if (buffer.getFirst() <= threshold) {
-				buffer.removeFirst();
-				counter++;
-			} else {
-				break;
-			}
-		}
-		return counter;
-
-	}
-
-	@Override
-	public TimeEvictionPolicy<DATA> clone() {
-		return new TimeEvictionPolicy<DATA>(granularity, timestampWrapper);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof TimeEvictionPolicy)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				TimeEvictionPolicy<DATA> otherPolicy = (TimeEvictionPolicy<DATA>) other;
-				return granularity == otherPolicy.granularity
-						&& timestampWrapper.equals(otherPolicy.timestampWrapper);
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-
-	public long getWindowSize() {
-		return granularity;
-	}
-
-	@Override
-	public String toString() {
-		return "TimePolicy(" + granularity + ", " + timestampWrapper.getClass().getSimpleName()
-				+ ")";
-	}
-
-	public TimestampWrapper<DATA> getTimeStampWrapper() {
-		return timestampWrapper;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicy.java
deleted file mode 100644
index 03984a9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TimeTriggerPolicy.java
+++ /dev/null
@@ -1,209 +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.flink.streaming.api.windowing.policy;
-
-import java.util.LinkedList;
-
-import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-
-/**
- * This trigger policy triggers with regard to the time. The is measured using a
- * given {@link Timestamp} implementation. A point in time is always represented
- * as long. Therefore, parameters such as granularity and delay can be set as
- * long value as well.
- * 
- * @param <DATA>
- *            The type of the incoming data points which are processed by this
- *            policy.
- */
-public class TimeTriggerPolicy<DATA> implements ActiveTriggerPolicy<DATA>,
-		CloneableTriggerPolicy<DATA>, CentralActiveTrigger<DATA> {
-
-	/**
-	 * auto generated version id
-	 */
-	private static final long serialVersionUID = -5122753802440196719L;
-
-	protected long startTime;
-	public long granularity;
-	public TimestampWrapper<DATA> timestampWrapper;
-
-	/**
-	 * This is mostly the same as
-	 * {@link TimeTriggerPolicy#TimeTriggerPolicy(long, TimestampWrapper)}. In addition
-	 * to granularity and timestamp a delay can be specified for the first
-	 * trigger. If the start time given by the timestamp is x, the delay is y,
-	 * and the granularity is z, the first trigger will happen at x+y+z.
-	 * 
-	 * @param granularity
-	 *            The granularity of the trigger. If this value is set to 2 the
-	 *            policy will trigger at every second time point
-	 * @param timestampWrapper
-	 *            The {@link TimestampWrapper} to measure the time with. This
-	 *            can be either user defined of provided by the API.
-	 */
-	public TimeTriggerPolicy(long granularity, TimestampWrapper<DATA> timestampWrapper) {
-		this.startTime = timestampWrapper.getStartTime();
-		this.timestampWrapper = timestampWrapper;
-		this.granularity = granularity;
-	}
-
-	/**
-	 * This method checks if we missed a window end. If this is the case we
-	 * trigger the missed windows using fake elements.
-	 */
-	@Override
-	public synchronized Object[] preNotifyTrigger(DATA datapoint) {
-		LinkedList<Object> fakeElements = new LinkedList<Object>();
-		// check if there is more then one window border missed
-		// use > here. In case >= would fit, the regular call will do the job.
-		while (timestampWrapper.getTimestamp(datapoint) >= startTime + granularity) {
-			startTime += granularity;
-			fakeElements.add(startTime - 1);
-		}
-		return (Object[]) fakeElements.toArray();
-	}
-
-	/**
-	 * In case {@link SystemTimestamp} is used, a runnable is returned which
-	 * triggers based on the current system time. If any other time measure is
-	 * used the method returns null.
-	 * 
-	 * @param callback
-	 *            The object which is takes the callbacks for adding fake
-	 *            elements out of the runnable.
-	 * @return A runnable is returned which triggers based on the current system
-	 *         time. If any other time measure is used the method return null.
-	 */
-	@Override
-	public Runnable createActiveTriggerRunnable(ActiveTriggerCallback callback) {
-		if (this.timestampWrapper.isDefaultTimestamp()) {
-			return new TimeCheck(callback);
-		} else {
-			return null;
-		}
-	}
-
-	/**
-	 * This method is only called in case the runnable triggers a window end
-	 * according to the {@link SystemTimestamp}.
-	 * 
-	 * @param callback
-	 *            The callback object.
-	 */
-	public synchronized Object activeFakeElementEmission(ActiveTriggerCallback callback) {
-
-		// start time is excluded, but end time is included: >=
-		if (System.currentTimeMillis() >= startTime + granularity) {
-			startTime += granularity;
-			if (callback != null) {
-				callback.sendFakeElement(startTime - 1);
-			}
-			return startTime - 1;
-		}
-		return null;
-
-	}
-
-	private class TimeCheck implements Runnable {
-		ActiveTriggerCallback callback;
-
-		public TimeCheck(ActiveTriggerCallback callback) {
-			this.callback = callback;
-		}
-
-		@Override
-		public void run() {
-			while (true) {
-				// wait for the specified granularity
-				try {
-					Thread.sleep(granularity);
-				} catch (InterruptedException e) {
-					// ignore it...
-				}
-				// Trigger using the respective methods. Methods are
-				// synchronized to prevent race conditions between real and fake
-				// elements at the policy.
-				activeFakeElementEmission(callback);
-			}
-		}
-	}
-
-	@Override
-	public synchronized boolean notifyTrigger(DATA datapoint) {
-		long recordTime = timestampWrapper.getTimestamp(datapoint);
-		if (recordTime >= startTime + granularity) {
-			if (granularity != 0) {
-				startTime = recordTime - ((recordTime - startTime) % granularity);
-			}
-			return true;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public TimeTriggerPolicy<DATA> clone() {
-		return new TimeTriggerPolicy<DATA>(granularity, timestampWrapper);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof TimeTriggerPolicy)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				TimeTriggerPolicy<DATA> otherPolicy = (TimeTriggerPolicy<DATA>) other;
-				return startTime == otherPolicy.startTime && granularity == otherPolicy.granularity
-						&& timestampWrapper.equals(otherPolicy.timestampWrapper);
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-
-	public long getSlideSize() {
-		return granularity;
-	}
-
-	@Override
-	public String toString() {
-		return "TimePolicy(" + granularity + ", " + timestampWrapper.getClass().getSimpleName()
-				+ ")";
-	}
-
-	public TimestampWrapper<DATA> getTimeStampWrapper() {
-		return timestampWrapper;
-	}
-
-	@Override
-	public Object[] notifyOnLastGlobalElement(DATA datapoint) {
-		LinkedList<Object> fakeElements = new LinkedList<Object>();
-		// check if there is more then one window border missed
-		// use > here. In case >= would fit, the regular call will do the job.
-		while (timestampWrapper.getTimestamp(datapoint) >= startTime + granularity) {
-			startTime += granularity;
-			fakeElements.add(startTime - 1);
-		}
-		return (Object[]) fakeElements.toArray();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TriggerPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TriggerPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TriggerPolicy.java
deleted file mode 100644
index c212df6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TriggerPolicy.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-import java.io.Serializable;
-
-/**
- * Proves and returns if a new window should be started. In case the trigger
- * occurs (return value true) the UDF will be executed on the current element
- * buffer without the last added element which is provided as parameter. This
- * element will be added to the buffer after the execution of the UDF.
- * 
- * @param <DATA>
- *            The data type which can be handled by this policy
- */
-public interface TriggerPolicy<DATA> extends Serializable {
-
-	/**
-	 * Proves and returns if a new window should be started. In case the trigger
-	 * occurs (return value true) the UDF will be executed on the current
-	 * element buffer without the last added element which is provided as
-	 * parameter. This element will be added to the buffer after the execution
-	 * of the UDF.
-	 * 
-	 * There are possibly different strategies for eviction and triggering: 1)
-	 * including last data point: Better/faster for count eviction 2) excluding
-	 * last data point: Essentially required for time based eviction and delta
-	 * rules As 2) is required for some policies and the benefit of using 1) is
-	 * small for the others, policies are implemented according to 2).
-	 *
-	 * @param datapoint
-	 *            the data point which arrived
-	 * @return true if the current windows should be closed, otherwise false. In
-	 *         true case the given data point will be part of the next window
-	 *         and will not be included in the current one.
-	 */
-	public boolean notifyTrigger(DATA datapoint);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicy.java
deleted file mode 100644
index 08c49e9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/TumblingEvictionPolicy.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-/**
- * This eviction policy deletes all elements from the buffer in case a trigger
- * occurred. Therefore, it is the default eviction policy to be used for any
- * tumbling window.
- * 
- * By default this policy does not react on fake elements. Wrap it in an
- * {@link ActiveEvictionPolicyWrapper} to make it clearing the buffer even on
- * fake elements.
- * 
- * @param <DATA>
- *            The type of the data points which is handled by this policy
- */
-public class TumblingEvictionPolicy<DATA> implements CloneableEvictionPolicy<DATA> {
-
-	/**
-	 * Auto generated version ID
-	 */
-	private static final long serialVersionUID = -4018019069267281155L;
-
-	/**
-	 * Counter for the current number of elements in the buffer
-	 */
-	private int counter = 0;
-
-	/**
-	 * This is the default constructor providing no special functionality. This
-	 * eviction policy deletes all elements from the buffer in case a trigger
-	 * occurred. Therefore, it is the default eviction policy to be used for any
-	 * tumbling window.
-	 */
-	public TumblingEvictionPolicy() {
-		// default constructor, no further logic needed
-	}
-
-	/**
-	 * This constructor allows to set a custom start value for the element
-	 * counter.
-	 * 
-	 * This eviction policy deletes all elements from the buffer in case a
-	 * trigger occurred. Therefore, it is the default eviction policy to be used
-	 * for any tumbling window.
-	 * 
-	 * @param startValue
-	 *            A start value for the element counter
-	 */
-	public TumblingEvictionPolicy(int startValue) {
-		this.counter = startValue;
-	}
-
-	/**
-	 * Deletes all elements from the buffer in case the trigger occurred.
-	 */
-	@Override
-	public int notifyEviction(Object datapoint, boolean triggered, int bufferSize) {
-		if (triggered) {
-			// The current data point will be part of the next window!
-			// Therefore the counter needs to be set to one already.
-			int tmpCounter = counter;
-			counter = 1;
-			return tmpCounter;
-		} else {
-			counter++;
-			return 0;
-		}
-	}
-
-	@Override
-	public TumblingEvictionPolicy<DATA> clone() {
-		return new TumblingEvictionPolicy<DATA>(counter);
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof TumblingEvictionPolicy)) {
-			return false;
-		} else {
-			return true;
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "TumblingPolicy";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java
deleted file mode 100644
index 33fb29d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/BasicWindowBuffer.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import java.util.LinkedList;
-import java.util.NoSuchElementException;
-
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-/**
- * Basic window buffer that stores the elements in a simple list without any
- * pre-aggregation.
- */
-public class BasicWindowBuffer<T> extends WindowBuffer<T> {
-
-	private static final long serialVersionUID = 1L;
-	protected LinkedList<T> buffer;
-
-	public BasicWindowBuffer() {
-		this.buffer = new LinkedList<T>();
-	}
-
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-		if (emitEmpty || !buffer.isEmpty()) {
-			StreamWindow<T> currentWindow = createEmptyWindow();
-			currentWindow.addAll(buffer);
-			collector.collect(new StreamRecord<StreamWindow<T>>(currentWindow));
-		} 
-	}
-
-	public void store(T element) throws Exception {
-		buffer.add(element);
-	}
-
-	public void evict(int n) {
-		for (int i = 0; i < n; i++) {
-			try {
-				buffer.removeFirst();
-			} catch (NoSuchElementException e) {
-				// In case no more elements are in the buffer:
-				// Prevent failure and stop deleting.
-				break;
-			}
-		}
-	}
-
-	@Override
-	public BasicWindowBuffer<T> clone() {
-		return new BasicWindowBuffer<T>();
-	}
-
-	@Override
-	public String toString() {
-		return buffer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java
deleted file mode 100644
index 195a966..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountGroupedPreReducer.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-public class JumpingCountGroupedPreReducer<T> extends TumblingGroupedPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final long countToSkip; // How many elements should be jumped over
-	private long skipped = 0; // How many elements have we skipped since the last emitWindow
-
-	public JumpingCountGroupedPreReducer(ReduceFunction<T> reducer, KeySelector<T, ?> keySelector,
-										TypeSerializer<T> serializer, long countToSkip) {
-		super(reducer, keySelector, serializer);
-		this.countToSkip = countToSkip;
-	}
-
-	@Override
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-		super.emitWindow(collector);
-		skipped = 0;
-	}
-
-	@Override
-	public void store(T element) throws Exception {
-		if(skipped == countToSkip){
-			super.store(element);
-		} else {
-			skipped++;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java
deleted file mode 100644
index 17fe408..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingCountPreReducer.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-/**
- * Non-grouped pre-reducer for jumping time eviction policy
- * (the policies are based on count, and the slide size is larger than the window size).
- */
-public class JumpingCountPreReducer<T> extends TumblingPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final long countToSkip; // How many elements should be jumped over
-	private long skipped = 0; // How many elements have we skipped since the last emitWindow
-
-	public JumpingCountPreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer, long countToSkip){
-		super(reducer, serializer);
-		this.countToSkip = countToSkip;
-	}
-
-	@Override
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-		super.emitWindow(collector);
-		skipped = 0;
-	}
-
-	@Override
-	public void store(T element) throws Exception {
-		if(skipped == countToSkip){
-			super.store(element);
-		} else {
-			skipped++;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java
deleted file mode 100644
index a92fc98..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimeGroupedPreReducer.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-public class JumpingTimeGroupedPreReducer<T> extends TumblingGroupedPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private TimestampWrapper<T> timestampWrapper;
-	protected long windowStartTime;
-	private long slideSize;
-
-	public JumpingTimeGroupedPreReducer(ReduceFunction<T> reducer, KeySelector<T, ?> keySelector,
-										TypeSerializer<T> serializer,
-										long slideSize, long windowSize, TimestampWrapper<T> timestampWrapper){
-		super(reducer, keySelector, serializer);
-		this.timestampWrapper = timestampWrapper;
-		this.windowStartTime = timestampWrapper.getStartTime() + slideSize - windowSize;
-		this.slideSize = slideSize;
-	}
-
-	@Override
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-		super.emitWindow(collector);
-		windowStartTime += slideSize;
-	}
-
-	public void store(T element) throws Exception {
-		if(timestampWrapper.getTimestamp(element) >= windowStartTime) {
-			super.store(element);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java
deleted file mode 100644
index 1a47bc8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducer.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-/**
- * Non-grouped pre-reducer for jumping time eviction policy
- * (the policies are based on time, and the slide size is larger than the window size).
- */
-public class JumpingTimePreReducer<T> extends TumblingPreReducer<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private TimestampWrapper<T> timestampWrapper;
-	protected long windowStartTime;
-	private long slideSize;
-
-	public JumpingTimePreReducer(ReduceFunction<T> reducer, TypeSerializer<T> serializer,
-								long slideSize, long windowSize, TimestampWrapper<T> timestampWrapper){
-		super(reducer, serializer);
-		this.timestampWrapper = timestampWrapper;
-		this.windowStartTime = timestampWrapper.getStartTime() + slideSize - windowSize;
-		this.slideSize = slideSize;
-	}
-
-	@Override
-	public void emitWindow(Collector<StreamRecord<StreamWindow<T>>> collector) {
-		super.emitWindow(collector);
-		windowStartTime += slideSize;
-	}
-
-	public void store(T element) throws Exception {
-		if(timestampWrapper.getTimestamp(element) >= windowStartTime) {
-			super.store(element);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/PreAggregator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/PreAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/PreAggregator.java
deleted file mode 100644
index 1b95248..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowbuffer/PreAggregator.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUStreamRecord<?>WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-/**
- * Interface for marking window pre-aggregators that fully process the window so
- * that no further reduce step is necessary afterwards.
- */
-public interface PreAggregator {
-
-}


[04/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTupleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTupleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTupleTest.java
new file mode 100644
index 0000000..c05f281
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldFromTupleTest.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple10;
+import org.apache.flink.api.java.tuple.Tuple11;
+import org.apache.flink.api.java.tuple.Tuple12;
+import org.apache.flink.api.java.tuple.Tuple13;
+import org.apache.flink.api.java.tuple.Tuple14;
+import org.apache.flink.api.java.tuple.Tuple15;
+import org.apache.flink.api.java.tuple.Tuple16;
+import org.apache.flink.api.java.tuple.Tuple17;
+import org.apache.flink.api.java.tuple.Tuple18;
+import org.apache.flink.api.java.tuple.Tuple19;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple20;
+import org.apache.flink.api.java.tuple.Tuple21;
+import org.apache.flink.api.java.tuple.Tuple22;
+import org.apache.flink.api.java.tuple.Tuple23;
+import org.apache.flink.api.java.tuple.Tuple24;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.tuple.Tuple8;
+import org.apache.flink.api.java.tuple.Tuple9;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class FieldFromTupleTest {
+
+	private String[] testStrings;
+
+	@Before
+	public void init() {
+		testStrings = new String[Tuple.MAX_ARITY];
+		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
+			testStrings[i] = Integer.toString(i);
+		}
+	}
+
+	@Test
+	public void testSingleFieldExtraction() throws InstantiationException, IllegalAccessException {
+		// extract single fields
+		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
+			Tuple current = (Tuple) CLASSES[i].newInstance();
+			for (int j = 0; j < i; j++) {
+				current.setField(testStrings[j], j);
+			}
+			for (int j = 0; j < i; j++) {
+				assertEquals(testStrings[j], new FieldFromTuple<String>(j).extract(current));
+			}
+		}
+	}
+
+	private static final Class<?>[] CLASSES = new Class<?>[] { Tuple1.class, Tuple2.class,
+			Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class,
+			Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class,
+			Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class,
+			Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class,
+			Tuple24.class, Tuple25.class };
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArrayTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArrayTest.java
new file mode 100644
index 0000000..7a9a716
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromArrayTest.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.flink.streaming.api.functions.windowing.delta.extractor.FieldsFromArray;
+import org.junit.Test;
+
+public class FieldsFromArrayTest {
+
+	String[] testStringArray = { "0", "1", "2", "3", "4" };
+	Integer[] testIntegerArray = { 10, 11, 12, 13, 14 };
+	int[] testIntArray = { 20, 21, 22, 23, 24 };
+
+	@Test
+	public void testStringArray() {
+		// check single field extraction
+		for (int i = 0; i < testStringArray.length; i++) {
+			String[] tmp = { testStringArray[i] };
+			arrayEqualityCheck(tmp,
+					new FieldsFromArray<String>(String.class, i).extract(testStringArray));
+		}
+
+		// check reverse order
+		String[] reverseOrder = new String[testStringArray.length];
+		for (int i = 0; i < testStringArray.length; i++) {
+			reverseOrder[i] = testStringArray[testStringArray.length - i - 1];
+		}
+		arrayEqualityCheck(reverseOrder,
+				new FieldsFromArray<String>(String.class, 4, 3, 2, 1, 0).extract(testStringArray));
+
+		// check picking fields and reorder
+		String[] crazyOrder = { testStringArray[4], testStringArray[1], testStringArray[2] };
+		arrayEqualityCheck(crazyOrder,
+				new FieldsFromArray<String>(String.class, 4, 1, 2).extract(testStringArray));
+	}
+
+	@Test
+	public void testIntegerArray() {
+		// check single field extraction
+		for (int i = 0; i < testIntegerArray.length; i++) {
+			Integer[] tmp = { testIntegerArray[i] };
+			arrayEqualityCheck(tmp,
+					new FieldsFromArray<Integer>(Integer.class, i).extract(testIntegerArray));
+		}
+
+		// check reverse order
+		Integer[] reverseOrder = new Integer[testIntegerArray.length];
+		for (int i = 0; i < testIntegerArray.length; i++) {
+			reverseOrder[i] = testIntegerArray[testIntegerArray.length - i - 1];
+		}
+		arrayEqualityCheck(reverseOrder,
+				new FieldsFromArray<Integer>(Integer.class, 4, 3, 2, 1, 0)
+						.extract(testIntegerArray));
+
+		// check picking fields and reorder
+		Integer[] crazyOrder = { testIntegerArray[4], testIntegerArray[1], testIntegerArray[2] };
+		arrayEqualityCheck(crazyOrder,
+				new FieldsFromArray<Integer>(Integer.class, 4, 1, 2).extract(testIntegerArray));
+
+	}
+
+	@Test
+	public void testIntArray() {
+		for (int i = 0; i < testIntArray.length; i++) {
+			Integer[] tmp = { testIntArray[i] };
+			arrayEqualityCheck(tmp,
+					new FieldsFromArray<Integer>(Integer.class, i).extract(testIntArray));
+		}
+
+		// check reverse order
+		Integer[] reverseOrder = new Integer[testIntArray.length];
+		for (int i = 0; i < testIntArray.length; i++) {
+			reverseOrder[i] = testIntArray[testIntArray.length - i - 1];
+		}
+		arrayEqualityCheck(reverseOrder,
+				new FieldsFromArray<Integer>(Integer.class, 4, 3, 2, 1, 0).extract(testIntArray));
+
+		// check picking fields and reorder
+		Integer[] crazyOrder = { testIntArray[4], testIntArray[1], testIntArray[2] };
+		arrayEqualityCheck(crazyOrder,
+				new FieldsFromArray<Integer>(Integer.class, 4, 1, 2).extract(testIntArray));
+
+	}
+
+	private void arrayEqualityCheck(Object[] array1, Object[] array2) {
+		assertEquals("The result arrays must have the same length", array1.length, array2.length);
+		for (int i = 0; i < array1.length; i++) {
+			assertEquals("Unequal fields at position " + i, array1[i], array2[i]);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTupleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTupleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTupleTest.java
new file mode 100644
index 0000000..025ed8a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTupleTest.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.flink.streaming.api.functions.windowing.delta.extractor;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple10;
+import org.apache.flink.api.java.tuple.Tuple11;
+import org.apache.flink.api.java.tuple.Tuple12;
+import org.apache.flink.api.java.tuple.Tuple13;
+import org.apache.flink.api.java.tuple.Tuple14;
+import org.apache.flink.api.java.tuple.Tuple15;
+import org.apache.flink.api.java.tuple.Tuple16;
+import org.apache.flink.api.java.tuple.Tuple17;
+import org.apache.flink.api.java.tuple.Tuple18;
+import org.apache.flink.api.java.tuple.Tuple19;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple20;
+import org.apache.flink.api.java.tuple.Tuple21;
+import org.apache.flink.api.java.tuple.Tuple22;
+import org.apache.flink.api.java.tuple.Tuple23;
+import org.apache.flink.api.java.tuple.Tuple24;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.tuple.Tuple8;
+import org.apache.flink.api.java.tuple.Tuple9;
+import org.apache.flink.streaming.api.functions.windowing.delta.extractor.FieldsFromTuple;
+import org.junit.Before;
+import org.junit.Test;
+
+public class FieldsFromTupleTest {
+
+	private double[] testDouble;
+
+	@Before
+	public void init() {
+		testDouble = new double[Tuple.MAX_ARITY];
+		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
+			testDouble[i] = i;
+		}
+	}
+
+	@Test
+	public void testUserSpecifiedOrder() throws InstantiationException, IllegalAccessException {
+		Tuple currentTuple = (Tuple) CLASSES[Tuple.MAX_ARITY - 1].newInstance();
+		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
+			currentTuple.setField(testDouble[i], i);
+		}
+
+		double[] expected = { testDouble[5], testDouble[3], testDouble[6], testDouble[7],
+				testDouble[0] };
+		arrayEqualityCheck(expected, new FieldsFromTuple(5, 3, 6, 7, 0).extract(currentTuple));
+
+		double[] expected2 = { testDouble[0], testDouble[Tuple.MAX_ARITY - 1] };
+		arrayEqualityCheck(expected2,
+				new FieldsFromTuple(0, Tuple.MAX_ARITY - 1).extract(currentTuple));
+
+		double[] expected3 = { testDouble[Tuple.MAX_ARITY - 1], testDouble[0] };
+		arrayEqualityCheck(expected3,
+				new FieldsFromTuple(Tuple.MAX_ARITY - 1, 0).extract(currentTuple));
+
+		double[] expected4 = { testDouble[13], testDouble[4], testDouble[5], testDouble[4],
+				testDouble[2], testDouble[8], testDouble[6], testDouble[2], testDouble[8],
+				testDouble[3], testDouble[5], testDouble[2], testDouble[16], testDouble[4],
+				testDouble[3], testDouble[2], testDouble[6], testDouble[4], testDouble[7],
+				testDouble[4], testDouble[2], testDouble[8], testDouble[7], testDouble[2] };
+		arrayEqualityCheck(expected4, new FieldsFromTuple(13, 4, 5, 4, 2, 8, 6, 2, 8, 3, 5, 2, 16,
+				4, 3, 2, 6, 4, 7, 4, 2, 8, 7, 2).extract(currentTuple));
+	}
+
+	private void arrayEqualityCheck(double[] array1, double[] array2) {
+		assertEquals("The result arrays must have the same length", array1.length, array2.length);
+		for (int i = 0; i < array1.length; i++) {
+			assertEquals("Unequal fields at position " + i, array1[i], array2[i], 0d);
+		}
+	}
+
+	private static final Class<?>[] CLASSES = new Class<?>[] { Tuple1.class, Tuple2.class,
+			Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class,
+			Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class,
+			Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class,
+			Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class,
+			Tuple24.class, Tuple25.class };
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
index c116c01..d00dc67 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
@@ -18,9 +18,7 @@
 package org.apache.flink.streaming.api.operators.co;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -29,17 +27,12 @@ import java.util.List;
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.util.Collector;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
@@ -184,15 +177,4 @@ public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
 
 		assertEquals(expected, result);
 	}
-
-	private static class IntegerTimestamp implements Timestamp<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long getTimestamp(Integer value) {
-			return value;
-		}
-
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizerTest.java
deleted file mode 100644
index c464b47..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/GroupedStreamDiscretizerTest.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.operators.windowing.GroupedStreamDiscretizer;
-import org.apache.flink.streaming.api.operators.windowing.GroupedWindowBuffer;
-import org.apache.flink.streaming.api.operators.windowing.StreamWindowBuffer;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.WindowEvent;
-import org.apache.flink.streaming.api.windowing.policy.CloneableEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.CloneableTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.CountTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TumblingEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBuffer;
-import org.apache.flink.streaming.util.MockContext;
-import org.junit.Test;
-
-public class GroupedStreamDiscretizerTest {
-
-	KeySelector<Tuple2<Integer, String>, ?> keySelector = new KeySelector<Tuple2<Integer, String>, String>() {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple2<Integer, String> value) throws Exception {
-			return value.f1;
-		}
-	};
-
-	/**
-	 * Test for not active distributed triggers with single field
-	 */
-	@Test
-	public void groupedDiscretizerTest() {
-
-		List<Integer> inputs = new ArrayList<Integer>();
-		inputs.add(1);
-		inputs.add(2);
-		inputs.add(2);
-		inputs.add(3);
-		inputs.add(4);
-		inputs.add(5);
-		inputs.add(10);
-		inputs.add(11);
-		inputs.add(11);
-
-		Set<StreamWindow<Integer>> expected = new HashSet<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(2, 2));
-		expected.add(StreamWindow.fromElements(1, 3));
-		expected.add(StreamWindow.fromElements(5, 11));
-		expected.add(StreamWindow.fromElements(4, 10));
-		expected.add(StreamWindow.fromElements(11));
-
-		KeySelector<Integer, Integer> keySelector = new KeySelector<Integer, Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Integer getKey(Integer value) {
-				return value % 2;
-			}
-		};
-
-		CloneableTriggerPolicy<Integer> trigger = new CountTriggerPolicy<Integer>(2);
-		CloneableEvictionPolicy<Integer> eviction = new TumblingEvictionPolicy<Integer>();
-
-		GroupedStreamDiscretizer<Integer> discretizer = new GroupedStreamDiscretizer<Integer>(
-				keySelector, trigger, eviction);
-
-		StreamWindowBuffer<Integer> buffer = new GroupedWindowBuffer<Integer>(
-				new BasicWindowBuffer<Integer>(), keySelector);
-
-		List<WindowEvent<Integer>> bufferEvents = MockContext.createAndExecute(discretizer,
-				inputs);
-		List<StreamWindow<Integer>> result = MockContext.createAndExecute(buffer, bufferEvents);
-
-		assertEquals(expected, new HashSet<StreamWindow<Integer>>(result));
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java
deleted file mode 100644
index 2c06c00..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeITCase.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.helper.Time;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests that {@link ParallelMerge} does not swallow records of the
- * last window.
- */
-public class ParallelMergeITCase extends StreamingProgramTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-	protected final String input = "To be, or not to be,--that is the question:--" +
-									"Whether 'tis nobler in the mind to suffer";
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", input);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		List<String> resultLines = new ArrayList<>();
-		readAllResultLines(resultLines, resultPath);
-
-		// check that result lines are not swallowed, as every element is expected to be in the
-		// last time window we either get the right output or no output at all
-		if (resultLines.isEmpty()){
-			Assert.fail();
-		}
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<String> text = env.fromElements(input);
-
-		DataStream<Tuple2<String, Integer>> counts =
-				text.flatMap(new Tokenizer())
-						.window(Time.of(1000, TimeUnit.MILLISECONDS))
-						.keyBy(0)
-						.sum(1)
-						.flatten();
-
-		counts.writeAsText(resultPath);
-
-		try {
-			env.execute();
-		} catch (RuntimeException e){
-			// might happen at closing the active window
-			// do nothing
-		}
-	}
-
-	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<Tuple2<String, Integer>> out)
-				throws Exception {
-			String[] tokens = value.toLowerCase().split("\\W+");
-
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(Tuple2.of(token, 1));
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java
deleted file mode 100644
index f111890..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/ParallelMergeTest.java
+++ /dev/null
@@ -1,122 +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.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class ParallelMergeTest {
-
-	@Test
-	public void nonGroupedTest() throws Exception {
-
-		ReduceFunction<Integer> reducer = new ReduceFunction<Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Integer reduce(Integer a, Integer b) throws Exception {
-				return a + b;
-			}
-		};
-
-		TestOutput<StreamWindow<Integer>> output = new TestOutput<StreamWindow<Integer>>();
-		TimestampedCollector<StreamWindow<Integer>> collector = new TimestampedCollector<StreamWindow<Integer>>(output);
-		List<StreamWindow<Integer>> result = output.getCollected();
-
-		ParallelMerge<Integer> merger = new ParallelMerge<Integer>(reducer);
-		merger.numberOfDiscretizers = 2;
-
-		merger.flatMap1(createTestWindow(1), collector);
-		merger.flatMap1(createTestWindow(1), collector);
-		merger.flatMap2(new Tuple2<Integer, Integer>(1, 1), collector);
-		assertTrue(result.isEmpty());
-		merger.flatMap2(new Tuple2<Integer, Integer>(1, 1), collector);
-		assertEquals(StreamWindow.fromElements(2), result.get(0));
-
-		merger.flatMap2(new Tuple2<Integer, Integer>(2, 2), collector);
-		merger.flatMap1(createTestWindow(2), collector);
-		merger.flatMap1(createTestWindow(2), collector);
-		merger.flatMap2(new Tuple2<Integer, Integer>(2, 1), collector);
-		assertEquals(1, result.size());
-		merger.flatMap1(createTestWindow(2), collector);
-		assertEquals(StreamWindow.fromElements(3), result.get(1));
-
-		// check error handling
-		merger.flatMap1(createTestWindow(3), collector);
-		merger.flatMap2(new Tuple2<Integer, Integer>(3, 1), collector);
-		merger.flatMap2(new Tuple2<Integer, Integer>(3, 1), collector);
-
-		merger.flatMap2(new Tuple2<Integer, Integer>(4, 1), collector);
-		merger.flatMap2(new Tuple2<Integer, Integer>(4, 1), collector);
-		merger.flatMap1(createTestWindow(4), collector);
-		try {
-			merger.flatMap1(createTestWindow(4), collector);
-			fail();
-		} catch (RuntimeException e) {
-			// Do nothing
-		}
-
-		ParallelMerge<Integer> merger2 = new ParallelMerge<Integer>(reducer);
-		merger2.numberOfDiscretizers = 2;
-		merger2.flatMap1(createTestWindow(0), collector);
-		merger2.flatMap1(createTestWindow(1), collector);
-		merger2.flatMap1(createTestWindow(1), collector);
-		merger2.flatMap2(new Tuple2<Integer, Integer>(1, 1), collector);
-		try {
-			merger2.flatMap2(new Tuple2<Integer, Integer>(1, 1), collector);
-			fail();
-		} catch (RuntimeException e) {
-			// Do nothing
-		}
-
-	}
-
-	@Test
-	public void groupedTest() throws Exception {
-
-		TestOutput<StreamWindow<Integer>> output = new TestOutput<StreamWindow<Integer>>();
-		TimestampedCollector<StreamWindow<Integer>> collector = new TimestampedCollector<StreamWindow<Integer>>(output);
-		List<StreamWindow<Integer>> result = output.getCollected();
-
-		ParallelMerge<Integer> merger = new ParallelGroupedMerge<Integer>();
-		merger.numberOfDiscretizers = 2;
-
-		merger.flatMap1(createTestWindow(1), collector);
-		merger.flatMap1(createTestWindow(1), collector);
-		merger.flatMap2(new Tuple2<Integer, Integer>(1, 1), collector);
-		assertTrue(result.isEmpty());
-		merger.flatMap2(new Tuple2<Integer, Integer>(1, 1), collector);
-		assertEquals(StreamWindow.fromElements(1, 1), result.get(0));
-	}
-
-	private StreamWindow<Integer> createTestWindow(Integer id) {
-		StreamWindow<Integer> ret = new StreamWindow<Integer>(id);
-		ret.add(1);
-		return ret;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizerTest.java
deleted file mode 100644
index 24251f1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizerTest.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.streaming.api.operators.windowing.StreamDiscretizer;
-import org.apache.flink.streaming.api.operators.windowing.StreamWindowBuffer;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.WindowEvent;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.api.windowing.policy.CountTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TimeEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TumblingEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBuffer;
-import org.apache.flink.streaming.util.MockContext;
-import org.junit.Test;
-
-public class StreamDiscretizerTest {
-
-	
-	@Test
-	public void testDiscretizer() {
-
-		List<Integer> inputs = new ArrayList<Integer>();
-		inputs.add(1);
-		inputs.add(2);
-		inputs.add(2);
-		inputs.add(3);
-		inputs.add(4);
-		inputs.add(5);
-		inputs.add(10);
-		inputs.add(11);
-		inputs.add(11);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1, 2, 2, 3, 4));
-		expected.add(StreamWindow.fromElements(3, 4, 5));
-		expected.add(StreamWindow.fromElements(5));
-		expected.add(StreamWindow.fromElements(10));
-		expected.add(StreamWindow.fromElements(10, 11, 11));
-
-		Timestamp<Integer> myTimeStamp = new Timestamp<Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-		};
-
-		TriggerPolicy<Integer> trigger = new TimeTriggerPolicy<Integer>(2L,
-				new TimestampWrapper<Integer>(myTimeStamp, 3));
-
-		EvictionPolicy<Integer> eviction = new TimeEvictionPolicy<Integer>(4L,
-				new TimestampWrapper<Integer>(myTimeStamp, 1));
-		
-		
-
-		StreamDiscretizer<Integer> discretizer = new StreamDiscretizer<Integer>(trigger, eviction);
-		StreamWindowBuffer<Integer> buffer = new StreamWindowBuffer<Integer>(new BasicWindowBuffer<Integer>());
-
-		List<WindowEvent<Integer>> bufferEvents = MockContext.createAndExecute(discretizer, inputs);
-		List<StreamWindow<Integer>> result = MockContext.createAndExecute(buffer, bufferEvents);
-		
-		assertEquals(expected, result);
-	}
-
-	@Test
-	public void testDiscretizer2() {
-
-		List<Integer> inputs = new ArrayList<Integer>();
-		inputs.add(1);
-		inputs.add(2);
-		inputs.add(2);
-		inputs.add(3);
-		inputs.add(4);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1, 2));
-		expected.add(StreamWindow.fromElements(2, 3));
-		expected.add(StreamWindow.fromElements(4));
-
-		TriggerPolicy<Integer> trigger = new CountTriggerPolicy<Integer>(2);
-
-		EvictionPolicy<Integer> eviction = new TumblingEvictionPolicy<Integer>();
-
-		StreamDiscretizer<Integer> discretizer = new StreamDiscretizer<Integer>(trigger, eviction);
-		StreamWindowBuffer<Integer> buffer = new StreamWindowBuffer<Integer>(new BasicWindowBuffer<Integer>());
-
-		List<WindowEvent<Integer>> bufferEvents = MockContext.createAndExecute(discretizer, inputs);
-		List<StreamWindow<Integer>> result = MockContext.createAndExecute(buffer, bufferEvents);
-		assertEquals(expected, result);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattenerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattenerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattenerTest.java
deleted file mode 100644
index dc6d0d6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFlattenerTest.java
+++ /dev/null
@@ -1,53 +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.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.util.MockContext;
-import org.junit.Test;
-
-public class WindowFlattenerTest {
-
-	@Test
-	public void test() {
-		OneInputStreamOperator<StreamWindow<Integer>, Integer> flattener = new WindowFlattener<Integer>();
-
-		StreamWindow<Integer> w1 = StreamWindow.fromElements(1, 2, 3);
-		StreamWindow<Integer> w2 = new StreamWindow<Integer>();
-
-		List<StreamWindow<Integer>> input = new ArrayList<StreamWindow<Integer>>();
-		input.add(w1);
-		input.add(w2);
-
-		List<Integer> expected = new ArrayList<Integer>();
-		expected.addAll(w1);
-		expected.addAll(w2);
-
-		List<Integer> output = MockContext.createAndExecute(flattener, input);
-
-		assertEquals(expected, output);
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFolderTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFolderTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFolderTest.java
deleted file mode 100644
index 3b54069..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowFolderTest.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.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.util.MockContext;
-import org.junit.Test;
-
-public class WindowFolderTest {
-
-	@Test
-	public void test() {
-		OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<String>> windowReducer = new WindowFolder<Integer,String>(
-				new FoldFunction<Integer, String>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public String fold(String accumulator, Integer value) throws Exception {
-						return accumulator + value.toString();
-					}
-				}, "");
-
-		List<StreamWindow<Integer>> input = new ArrayList<StreamWindow<Integer>>();
-		input.add(StreamWindow.fromElements(1, 2, 3));
-		input.add(new StreamWindow<Integer>());
-		input.add(StreamWindow.fromElements(-1));
-
-		List<StreamWindow<String>> expected = new ArrayList<StreamWindow<String>>();
-		expected.add(StreamWindow.fromElements("123"));
-		expected.add(new StreamWindow<String>());
-		expected.add(StreamWindow.fromElements("-1"));
-
-		List<StreamWindow<String>> output = MockContext.createAndExecute(windowReducer, input);
-
-		assertEquals(expected, output);
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMapperTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMapperTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMapperTest.java
deleted file mode 100644
index 9836a99..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMapperTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.util.MockContext;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-public class WindowMapperTest {
-
-	@Test
-	public void test() {
-		OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<Integer>> windowMapper = new WindowMapper<Integer, Integer>(
-				new WindowMapFunction<Integer, Integer>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void mapWindow(Iterable<Integer> values, Collector<Integer> out)
-							throws Exception {
-						for (Integer v : values) {
-							out.collect(v);
-						}
-					}
-				});
-
-		List<StreamWindow<Integer>> input = new ArrayList<StreamWindow<Integer>>();
-		input.add(StreamWindow.fromElements(1, 2, 3));
-		input.add(new StreamWindow<Integer>());
-
-		List<StreamWindow<Integer>> output = MockContext.createAndExecute(windowMapper, input);
-
-		assertEquals(input, output);
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMergerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMergerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMergerTest.java
deleted file mode 100644
index 43e3785..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowMergerTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.util.MockContext;
-import org.junit.Test;
-
-public class WindowMergerTest {
-
-	@Test
-	public void test() throws Exception {
-		OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<Integer>> windowMerger = new WindowMerger<Integer>();
-
-		StreamWindow<Integer> w1 = new StreamWindow<Integer>();
-		StreamWindow<Integer> w2 = StreamWindow.fromElements(1, 2, 3, 4);
-		StreamWindow<Integer> w3 = StreamWindow.fromElements(-1, 2, 3, 4);
-		StreamWindow<Integer> w4_1 = new StreamWindow<Integer>(1, 2);
-		StreamWindow<Integer> w4_2 = new StreamWindow<Integer>(1, 2);
-		w4_1.add(1);
-		w4_2.add(2);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(w1);
-		expected.add(w2);
-		expected.add(w3);
-		expected.add(StreamWindow.fromElements(1, 2));
-
-		List<StreamWindow<Integer>> input = new ArrayList<StreamWindow<Integer>>();
-		input.add(w1);
-		input.add(w4_1);
-		input.addAll(StreamWindow.split(w2, 2));
-		input.addAll(StreamWindow.partitionBy(w3, new KeySelector<Integer, Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Integer getKey(Integer value) throws Exception {
-				return value % 2;
-			}
-		}, false));
-		input.add(w4_2);
-
-		List<StreamWindow<Integer>> output = MockContext.createAndExecute(windowMerger, input);
-
-		assertEquals(expected.size(), expected.size());
-		for (int i = 0; i < output.size(); i++) {
-			assertEquals(new HashSet<Integer>(expected.get(i)), new HashSet<Integer>(output.get(i)));
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitionerTest.java
deleted file mode 100644
index 7521a2b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowPartitionerTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.util.MockContext;
-import org.junit.Test;
-
-public class WindowPartitionerTest {
-
-	@Test
-	public void test() throws Exception {
-		OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<Integer>> splitPartitioner = new WindowPartitioner<Integer>(
-				2);
-
-		OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<Integer>> gbPartitioner = new WindowPartitioner<Integer>(
-				new MyKey());
-
-		StreamWindow<Integer> w1 = new StreamWindow<Integer>();
-		StreamWindow<Integer> w2 = StreamWindow.fromElements(1, 2, 3, 4);
-
-		List<StreamWindow<Integer>> expected1 = new ArrayList<StreamWindow<Integer>>();
-		expected1.addAll(StreamWindow.split(w1,2));
-		expected1.addAll(StreamWindow.split(w2,2));
-
-		List<StreamWindow<Integer>> expected2 = new ArrayList<StreamWindow<Integer>>();
-		expected2.addAll(StreamWindow.partitionBy(w1,new MyKey(),false));
-		expected2.addAll(StreamWindow.partitionBy(w2,new MyKey(),false));
-
-		List<StreamWindow<Integer>> input = new ArrayList<StreamWindow<Integer>>();
-		input.add(w1);
-		input.add(w2);
-
-		List<StreamWindow<Integer>> output1 = MockContext.createAndExecute(splitPartitioner, input);
-		List<StreamWindow<Integer>> output2 = MockContext.createAndExecute(gbPartitioner, input);
-
-		assertEquals(expected1, output1);
-		assertEquals(expected2, output2);
-
-	}
-
-	private static class MyKey implements KeySelector<Integer, Object> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Object getKey(Integer value) throws Exception {
-			return value / 2;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowReducerTest.java
deleted file mode 100644
index b78a5ba..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowReducerTest.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.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.util.MockContext;
-import org.junit.Test;
-
-public class WindowReducerTest {
-
-	@Test
-	public void test() {
-		OneInputStreamOperator<StreamWindow<Integer>, StreamWindow<Integer>> windowReducer = new WindowReducer<Integer>(
-				new ReduceFunction<Integer>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public Integer reduce(Integer value1, Integer value2) throws Exception {
-						return value1 + value2;
-					}
-				});
-
-		List<StreamWindow<Integer>> input = new ArrayList<StreamWindow<Integer>>();
-		input.add(StreamWindow.fromElements(1, 2, 3));
-		input.add(new StreamWindow<Integer>());
-		input.add(StreamWindow.fromElements(-1));
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(6));
-		expected.add(new StreamWindow<Integer>());
-		expected.add(StreamWindow.fromElements(-1));
-
-		List<StreamWindow<Integer>> output = MockContext.createAndExecute(windowReducer, input);
-
-		assertEquals(expected, output);
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
deleted file mode 100644
index 4611966..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/windowing/WindowingITCase.java
+++ /dev/null
@@ -1,529 +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.flink.streaming.api.operators.windowing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.helper.Count;
-import org.apache.flink.streaming.api.windowing.helper.FullStream;
-import org.apache.flink.streaming.api.windowing.helper.Time;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-public class WindowingITCase extends StreamingMultipleProgramsTestBase {
-
-	@SuppressWarnings("serial")
-	public static class ModKey implements KeySelector<Integer, Integer> {
-		private int m;
-
-		public ModKey(int m) {
-			this.m = m;
-		}
-
-		@Override
-		public Integer getKey(Integer value) throws Exception {
-			return value % m;
-		}
-	}
-
-	@SuppressWarnings("serial")
-	public static class IdentityWindowMap implements
-			WindowMapFunction<Integer, StreamWindow<Integer>> {
-
-		@Override
-		public void mapWindow(Iterable<Integer> values, Collector<StreamWindow<Integer>> out)
-				throws Exception {
-
-			StreamWindow<Integer> window = new StreamWindow<Integer>();
-
-			for (Integer value : values) {
-				window.add(value);
-			}
-			out.collect(window);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	@Test
-	public void test() throws Exception {
-
-		List<Integer> inputs = new ArrayList<Integer>();
-		inputs.add(1);
-		inputs.add(2);
-		inputs.add(2);
-		inputs.add(3);
-		inputs.add(4);
-		inputs.add(5);
-		inputs.add(10);
-		inputs.add(11);
-		inputs.add(11);
-
-		KeySelector<Integer, ?> key = new ModKey(2);
-
-		Timestamp<Integer> ts = new IntegerTimestamp();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(2);
-		env.disableOperatorChaining();
-
-		DataStream<Integer> source = env.fromCollection(inputs);
-
-		source.window(Time.of(3, ts, 1)).every(Time.of(2, ts, 1)).sum(0).getDiscretizedStream()
-				.addSink(new TestSink1());
-
-		source.window(Time.of(4, ts, 1)).keyBy(new ModKey(2))
-				.mapWindow(new IdentityWindowMap())
-				.flatten()
-				.addSink(new TestSink2()).name("TESTSIUNK2");
-
-		source.keyBy(key).window(Time.of(4, ts, 1)).sum(0).getDiscretizedStream()
-				.addSink(new TestSink4());
-
-		source.keyBy(new ModKey(3)).window(Count.of(2)).keyBy(new ModKey(2))
-				.mapWindow(new IdentityWindowMap())
-				.flatten()
-				.addSink(new TestSink5());
-
-		source.window(Time.of(2, ts)).every(Time.of(3, ts)).min(0).getDiscretizedStream()
-				.addSink(new TestSink3());
-
-		source.keyBy(key).window(Time.of(4, ts, 1)).max(0).getDiscretizedStream()
-				.addSink(new TestSink6());
-
-		source.window(Time.of(5, ts, 1)).mapWindow(new IdentityWindowMap())
-				.flatten()
-				.addSink(new TestSink7());
-
-		source.window(Time.of(5, ts, 1)).every(Time.of(4, ts, 1)).keyBy(new ModKey(2)).sum(0)
-				.getDiscretizedStream()
-				.addSink(new TestSink8());
-
-		try {
-			source.window(FullStream.window()).every(Count.of(2)).getDiscretizedStream();
-			fail();
-		} catch (Exception e) {
-		}
-		try {
-			source.window(FullStream.window()).getDiscretizedStream();
-			fail();
-		} catch (Exception e) {
-		}
-		try {
-			source.every(Count.of(5)).mapWindow(new IdentityWindowMap()).getDiscretizedStream();
-			fail();
-		} catch (Exception e) {
-		}
-
-		source.every(Count.of(4)).sum(0).getDiscretizedStream().addSink(new TestSink11());
-
-		source.window(FullStream.window()).every(Count.of(4)).keyBy(key).sum(0)
-				.getDiscretizedStream()
-				.addSink(new TestSink12());
-
-		DataStream<Integer> source2 = env.addSource(new ParallelSourceFunction<Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-				for (int i = 1; i <= 10; i++) {
-					ctx.collect(i);
-				}
-			}
-
-			@Override
-			public void cancel() {
-			}
-		});
-
-		DataStream<Integer> source3 = env.addSource(new RichParallelSourceFunction<Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			private int i = 1;
-
-			@Override
-			public void open(Configuration parameters) throws Exception {
-				super.open(parameters);
-				i = 1 + getRuntimeContext().getIndexOfThisSubtask();
-			}
-
-			@Override
-			public void cancel() {
-			}
-
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-				for (;i < 11; i += 2) {
-					ctx.collect(i);
-				}
-
-			}
-		});
-
-		source2.window(Time.of(2, ts, 1)).sum(0).getDiscretizedStream().addSink(new TestSink9());
-
-		source3.window(Time.of(5, ts, 1)).keyBy(new ModKey(2)).sum(0).getDiscretizedStream()
-				.addSink(new TestSink10());
-
-		source
-				.map(new MapFunction<Integer, Integer>() {
-					@Override
-					public Integer map(Integer value) throws Exception {
-						return value;
-					}
-				})
-				.every(Time.of(5, ts, 1)).sum(0).getDiscretizedStream()
-				.addSink(new TestSink13());
-
-		env.execute();
-
-		// sum ( Time of 3 slide 2 )
-		List<StreamWindow<Integer>> expected1 = new ArrayList<StreamWindow<Integer>>();
-		expected1.add(StreamWindow.fromElements(5));
-		expected1.add(StreamWindow.fromElements(11));
-		expected1.add(StreamWindow.fromElements(9));
-		expected1.add(StreamWindow.fromElements(10));
-		expected1.add(StreamWindow.fromElements(32));
-
-		validateOutput(expected1, TestSink1.windows);
-
-		// Tumbling Time of 4 grouped by mod 2
-		List<StreamWindow<Integer>> expected2 = new ArrayList<StreamWindow<Integer>>();
-		expected2.add(StreamWindow.fromElements(2, 2, 4));
-		expected2.add(StreamWindow.fromElements(1, 3));
-		expected2.add(StreamWindow.fromElements(5));
-		expected2.add(StreamWindow.fromElements(10));
-		expected2.add(StreamWindow.fromElements(11, 11));
-
-		validateOutput(expected2, TestSink2.windows);
-
-		// groupby mod 2 sum ( Tumbling Time of 4)
-		List<StreamWindow<Integer>> expected3 = new ArrayList<StreamWindow<Integer>>();
-		expected3.add(StreamWindow.fromElements(4));
-		expected3.add(StreamWindow.fromElements(5));
-		expected3.add(StreamWindow.fromElements(22));
-		expected3.add(StreamWindow.fromElements(8));
-		expected3.add(StreamWindow.fromElements(10));
-
-		validateOutput(expected3, TestSink4.windows);
-
-		// groupby mod3 Tumbling Count of 2 grouped by mod 2
-		List<StreamWindow<Integer>> expected4 = new ArrayList<StreamWindow<Integer>>();
-		expected4.add(StreamWindow.fromElements(2, 2));
-		expected4.add(StreamWindow.fromElements(1));
-		expected4.add(StreamWindow.fromElements(4));
-		expected4.add(StreamWindow.fromElements(5, 11));
-		expected4.add(StreamWindow.fromElements(10));
-		expected4.add(StreamWindow.fromElements(11));
-		expected4.add(StreamWindow.fromElements(3));
-
-		validateOutput(expected4, TestSink5.windows);
-
-		// min ( Time of 2 slide 3 )
-		List<StreamWindow<Integer>> expected5 = new ArrayList<StreamWindow<Integer>>();
-		expected5.add(StreamWindow.fromElements(1));
-		expected5.add(StreamWindow.fromElements(4));
-		expected5.add(StreamWindow.fromElements(10));
-
-		validateOutput(expected5, TestSink3.windows);
-
-		// groupby mod 2 max ( Tumbling Time of 4)
-		List<StreamWindow<Integer>> expected6 = new ArrayList<StreamWindow<Integer>>();
-		expected6.add(StreamWindow.fromElements(3));
-		expected6.add(StreamWindow.fromElements(5));
-		expected6.add(StreamWindow.fromElements(11));
-		expected6.add(StreamWindow.fromElements(4));
-		expected6.add(StreamWindow.fromElements(10));
-
-		validateOutput(expected6, TestSink6.windows);
-
-		List<StreamWindow<Integer>> expected7 = new ArrayList<StreamWindow<Integer>>();
-		expected7.add(StreamWindow.fromElements(1, 2, 2, 3, 4, 5));
-		expected7.add(StreamWindow.fromElements(10));
-		expected7.add(StreamWindow.fromElements(10, 11, 11));
-
-		validateOutput(expected7, TestSink7.windows);
-
-		List<StreamWindow<Integer>> expected8 = new ArrayList<StreamWindow<Integer>>();
-		expected8.add(StreamWindow.fromElements(4, 8));
-		expected8.add(StreamWindow.fromElements(4, 5));
-		expected8.add(StreamWindow.fromElements(10, 22));
-
-		for (List<Integer> sw : TestSink8.windows) {
-			Collections.sort(sw);
-		}
-
-		validateOutput(expected8, TestSink8.windows);
-
-		List<StreamWindow<Integer>> expected9 = new ArrayList<StreamWindow<Integer>>();
-		expected9.add(StreamWindow.fromElements(6));
-		expected9.add(StreamWindow.fromElements(14));
-		expected9.add(StreamWindow.fromElements(22));
-		expected9.add(StreamWindow.fromElements(30));
-		expected9.add(StreamWindow.fromElements(38));
-
-		validateOutput(expected9, TestSink9.windows);
-
-		List<StreamWindow<Integer>> expected10 = new ArrayList<StreamWindow<Integer>>();
-		expected10.add(StreamWindow.fromElements(6, 9));
-		expected10.add(StreamWindow.fromElements(16, 24));
-
-		for (List<Integer> sw : TestSink10.windows) {
-			Collections.sort(sw);
-		}
-
-		validateOutput(expected10, TestSink10.windows);
-
-		List<StreamWindow<Integer>> expected11 = new ArrayList<StreamWindow<Integer>>();
-		expected11.add(StreamWindow.fromElements(8));
-		expected11.add(StreamWindow.fromElements(38));
-		expected11.add(StreamWindow.fromElements(49));
-
-		for (List<Integer> sw : TestSink11.windows) {
-			Collections.sort(sw);
-		}
-
-		validateOutput(expected11, TestSink11.windows);
-
-		List<StreamWindow<Integer>> expected12 = new ArrayList<StreamWindow<Integer>>();
-		expected12.add(StreamWindow.fromElements(4, 4));
-		expected12.add(StreamWindow.fromElements(18, 20));
-		expected12.add(StreamWindow.fromElements(18, 31));
-
-		for (List<Integer> sw : TestSink12.windows) {
-			Collections.sort(sw);
-		}
-
-		validateOutput(expected12, TestSink12.windows);
-
-		List<StreamWindow<Integer>> expected13 = new ArrayList<StreamWindow<Integer>>();
-		expected13.add(StreamWindow.fromElements(17));
-		expected13.add(StreamWindow.fromElements(27));
-		expected13.add(StreamWindow.fromElements(49));
-
-		for (List<Integer> sw : TestSink13.windows) {
-			Collections.sort(sw);
-		}
-
-		validateOutput(expected13, TestSink13.windows);
-
-	}
-
-	public static <R> void validateOutput(List<R> expected, List<R> actual) {
-		assertEquals(new HashSet<R>(expected), new HashSet<R>(actual));
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink1 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink2 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink3 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink4 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink5 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink6 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink7 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink8 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink9 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink10 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink11 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink12 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	@SuppressWarnings("serial")
-	private static class TestSink13 implements SinkFunction<StreamWindow<Integer>> {
-
-		public static List<StreamWindow<Integer>> windows = Collections
-				.synchronizedList(new ArrayList<StreamWindow<Integer>>());
-
-		@Override
-		public void invoke(StreamWindow<Integer> value) throws Exception {
-			windows.add(value);
-		}
-
-	}
-
-	private static class IntegerTimestamp implements Timestamp<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long getTimestamp(Integer value) {
-			return value;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTest.java
deleted file mode 100644
index c3efc7b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTest.java
+++ /dev/null
@@ -1,201 +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.flink.streaming.api.windowing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.StreamWindowSerializer;
-import org.junit.Test;
-
-public class StreamWindowTest {
-
-	@Test
-	public void creationTest() {
-
-		StreamWindow<Integer> window1 = new StreamWindow<Integer>();
-		assertTrue(window1.isEmpty());
-		assertTrue(window1.windowID != 0);
-
-		window1.add(10);
-		assertEquals(1, window1.size());
-
-		StreamWindow<Integer> window2 = new StreamWindow<Integer>(window1);
-
-		assertTrue(window1.windowID == window2.windowID);
-		assertEquals(1, window2.size());
-
-		StreamWindow<Integer> window3 = new StreamWindow<Integer>(100);
-		assertEquals(100, window3.windowID);
-
-		StreamWindow<Integer> window4 = new StreamWindow<Integer>();
-		assertFalse(window4.windowID == window1.windowID);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void mergeTest() throws IOException {
-		StreamWindow<Integer> window1 = new StreamWindow<Integer>().setNumberOfParts(3);
-		StreamWindow<Integer> window2 = new StreamWindow<Integer>(window1.windowID, 3);
-		StreamWindow<Integer> window3 = new StreamWindow<Integer>(window1.windowID, 3);
-
-		window1.add(1);
-		window2.add(2);
-		window3.add(3);
-
-		Set<Integer> values = new HashSet<Integer>();
-		values.add(1);
-		values.add(2);
-		values.add(3);
-
-		StreamWindow<Integer> merged = StreamWindow.merge(window1, window2, window3);
-
-		assertEquals(3, merged.size());
-		assertEquals(window1.windowID, merged.windowID);
-		assertEquals(values, new HashSet<Integer>(merged));
-
-		try {
-			StreamWindow.merge(window1, new StreamWindow<Integer>());
-			fail();
-		} catch (RuntimeException e) {
-			// good
-		}
-
-		List<StreamWindow<Integer>> wList = StreamWindow.split(merged,3);
-
-		StreamWindow<Integer> merged2 = StreamWindow.merge(wList);
-
-		assertEquals(3, merged2.size());
-		assertEquals(window1.windowID, merged2.windowID);
-		assertEquals(values, new HashSet<Integer>(merged2));
-
-	}
-
-	@Test
-	public void serializerTest() throws IOException {
-
-		StreamWindow<Integer> streamWindow = new StreamWindow<Integer>();
-		streamWindow.add(1);
-		streamWindow.add(2);
-		streamWindow.add(3);
-
-		TypeSerializer<StreamWindow<Integer>> ts = new StreamWindowSerializer<Integer>(
-				BasicTypeInfo.INT_TYPE_INFO, null);
-
-		TestOutputView ow = new TestOutputView();
-
-		ts.serialize(streamWindow, ow);
-
-		TestInputView iw = ow.getInputView();
-
-		assertEquals(streamWindow, ts.deserialize(iw));
-
-	}
-
-	@Test
-	public void partitionTest() {
-		StreamWindow<Integer> streamWindow = new StreamWindow<Integer>();
-		streamWindow.add(1);
-		streamWindow.add(2);
-		streamWindow.add(3);
-		streamWindow.add(4);
-		streamWindow.add(5);
-		streamWindow.add(6);
-
-		List<StreamWindow<Integer>> split = StreamWindow.split(streamWindow,2);
-		assertEquals(2, split.size());
-		assertEquals(StreamWindow.fromElements(1, 2, 3), split.get(0));
-		assertEquals(StreamWindow.fromElements(4, 5, 6), split.get(1));
-
-		List<StreamWindow<Integer>> split2 = StreamWindow.split(streamWindow,6);
-		assertEquals(6, split2.size());
-		assertEquals(StreamWindow.fromElements(1), split2.get(0));
-		assertEquals(StreamWindow.fromElements(2), split2.get(1));
-		assertEquals(StreamWindow.fromElements(3), split2.get(2));
-		assertEquals(StreamWindow.fromElements(4), split2.get(3));
-		assertEquals(StreamWindow.fromElements(5), split2.get(4));
-		assertEquals(StreamWindow.fromElements(6), split2.get(5));
-
-		List<StreamWindow<Integer>> split3 = StreamWindow.split(streamWindow,10);
-		assertEquals(6, split3.size());
-		assertEquals(StreamWindow.fromElements(1), split3.get(0));
-		assertEquals(StreamWindow.fromElements(2), split3.get(1));
-		assertEquals(StreamWindow.fromElements(3), split3.get(2));
-		assertEquals(StreamWindow.fromElements(4), split3.get(3));
-		assertEquals(StreamWindow.fromElements(5), split3.get(4));
-		assertEquals(StreamWindow.fromElements(6), split3.get(5));
-
-	}
-
-	private class TestOutputView extends DataOutputStream implements DataOutputView {
-
-		public TestOutputView() {
-			super(new ByteArrayOutputStream(4096));
-		}
-
-		public TestInputView getInputView() {
-			ByteArrayOutputStream baos = (ByteArrayOutputStream) out;
-			return new TestInputView(baos.toByteArray());
-		}
-
-		@Override
-		public void skipBytesToWrite(int numBytes) throws IOException {
-			for (int i = 0; i < numBytes; i++) {
-				write(0);
-			}
-		}
-
-		@Override
-		public void write(DataInputView source, int numBytes) throws IOException {
-			byte[] buffer = new byte[numBytes];
-			source.readFully(buffer);
-			write(buffer);
-		}
-	}
-
-	private class TestInputView extends DataInputStream implements DataInputView {
-
-		public TestInputView(byte[] data) {
-			super(new ByteArrayInputStream(data));
-		}
-
-		@Override
-		public void skipBytesToRead(int numBytes) throws IOException {
-			while (numBytes > 0) {
-				int skipped = skipBytes(numBytes);
-				numBytes -= skipped;
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java
deleted file mode 100644
index 2e98a8f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing;
-
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-import org.apache.flink.api.java.typeutils.PojoField;
-import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-import java.util.ArrayList;
-
-import static org.junit.Assert.*;
-
-public class StreamWindowTypeInfoTest extends TestLogger {
-
-	public static class TestClass{}
-
-	@Test
-	public void testStreamWindowTypeInfoEquality() {
-		StreamWindowTypeInfo<TestClass> tpeInfo1 = new StreamWindowTypeInfo<>(new GenericTypeInfo<>(TestClass.class));
-		StreamWindowTypeInfo<TestClass> tpeInfo2 = new StreamWindowTypeInfo<>(new GenericTypeInfo<>(TestClass.class));
-
-		assertEquals(tpeInfo1, tpeInfo2);
-		assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-	}
-
-	@Test
-	public void testStreamWindowTypeInfoInequality() {
-		StreamWindowTypeInfo<TestClass> tpeInfo1 = new StreamWindowTypeInfo<>(new GenericTypeInfo<>(TestClass.class));
-		StreamWindowTypeInfo<TestClass> tpeInfo2 = new StreamWindowTypeInfo<>(new PojoTypeInfo<>(TestClass.class, new ArrayList<PojoField>()));
-
-		assertNotEquals(tpeInfo1, tpeInfo2);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTupleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTupleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTupleTest.java
deleted file mode 100644
index 17d3974..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTupleTest.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.extractor;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple10;
-import org.apache.flink.api.java.tuple.Tuple11;
-import org.apache.flink.api.java.tuple.Tuple12;
-import org.apache.flink.api.java.tuple.Tuple13;
-import org.apache.flink.api.java.tuple.Tuple14;
-import org.apache.flink.api.java.tuple.Tuple15;
-import org.apache.flink.api.java.tuple.Tuple16;
-import org.apache.flink.api.java.tuple.Tuple17;
-import org.apache.flink.api.java.tuple.Tuple18;
-import org.apache.flink.api.java.tuple.Tuple19;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple20;
-import org.apache.flink.api.java.tuple.Tuple21;
-import org.apache.flink.api.java.tuple.Tuple22;
-import org.apache.flink.api.java.tuple.Tuple23;
-import org.apache.flink.api.java.tuple.Tuple24;
-import org.apache.flink.api.java.tuple.Tuple25;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple6;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.tuple.Tuple8;
-import org.apache.flink.api.java.tuple.Tuple9;
-import org.apache.flink.streaming.api.windowing.extractor.ArrayFromTuple;
-import org.junit.Before;
-import org.junit.Test;
-
-public class ArrayFromTupleTest {
-
-	private String[] testStrings;
-
-	@Before
-	public void init() {
-		testStrings = new String[Tuple.MAX_ARITY];
-		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
-			testStrings[i] = Integer.toString(i);
-		}
-	}
-
-	@Test
-	public void testConvertFromTupleToArray() throws InstantiationException, IllegalAccessException {
-		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
-			Tuple currentTuple = (Tuple) CLASSES[i].newInstance();
-			String[] currentArray = new String[i + 1];
-			for (int j = 0; j <= i; j++) {
-				currentTuple.setField(testStrings[j], j);
-				currentArray[j] = testStrings[j];
-			}
-			arrayEqualityCheck(currentArray, new ArrayFromTuple().extract(currentTuple));
-		}
-	}
-
-	@Test
-	public void testUserSpecifiedOrder() throws InstantiationException, IllegalAccessException {
-		Tuple currentTuple = (Tuple) CLASSES[Tuple.MAX_ARITY - 1].newInstance();
-		for (int i = 0; i < Tuple.MAX_ARITY; i++) {
-			currentTuple.setField(testStrings[i], i);
-		}
-
-		String[] expected = { testStrings[5], testStrings[3], testStrings[6], testStrings[7],
-				testStrings[0] };
-		arrayEqualityCheck(expected, new ArrayFromTuple(5, 3, 6, 7, 0).extract(currentTuple));
-
-		String[] expected2 = { testStrings[0], testStrings[Tuple.MAX_ARITY - 1] };
-		arrayEqualityCheck(expected2,
-				new ArrayFromTuple(0, Tuple.MAX_ARITY - 1).extract(currentTuple));
-
-		String[] expected3 = { testStrings[Tuple.MAX_ARITY - 1], testStrings[0] };
-		arrayEqualityCheck(expected3,
-				new ArrayFromTuple(Tuple.MAX_ARITY - 1, 0).extract(currentTuple));
-
-		String[] expected4 = { testStrings[13], testStrings[4], testStrings[5], testStrings[4],
-				testStrings[2], testStrings[8], testStrings[6], testStrings[2], testStrings[8],
-				testStrings[3], testStrings[5], testStrings[2], testStrings[16], testStrings[4],
-				testStrings[3], testStrings[2], testStrings[6], testStrings[4], testStrings[7],
-				testStrings[4], testStrings[2], testStrings[8], testStrings[7], testStrings[2] };
-		arrayEqualityCheck(expected4, new ArrayFromTuple(13, 4, 5, 4, 2, 8, 6, 2, 8, 3, 5, 2, 16,
-				4, 3, 2, 6, 4, 7, 4, 2, 8, 7, 2).extract(currentTuple));
-	}
-
-	private void arrayEqualityCheck(Object[] array1, Object[] array2) {
-		assertEquals("The result arrays must have the same length", array1.length, array2.length);
-		for (int i = 0; i < array1.length; i++) {
-			assertEquals("Unequal fields at position " + i, array1[i], array2[i]);
-		}
-	}
-
-	private static final Class<?>[] CLASSES = new Class<?>[] { Tuple1.class, Tuple2.class,
-			Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class,
-			Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class,
-			Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class,
-			Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class,
-			Tuple24.class, Tuple25.class };
-}


[07/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java
deleted file mode 100644
index 529850f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java
+++ /dev/null
@@ -1,148 +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,
- * WITHOUStreamRecord<?>WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing;
-
-import java.io.IOException;
-
-import com.google.common.base.Preconditions;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-public final class StreamWindowSerializer<T> extends TypeSerializer<StreamWindow<T>> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final TypeSerializer<T> typeSerializer;
-	TypeSerializer<Integer> intSerializer = IntSerializer.INSTANCE;
-	TypeSerializer<Boolean> boolSerializer = BooleanSerializer.INSTANCE;
-
-	public StreamWindowSerializer(TypeInformation<T> typeInfo, ExecutionConfig conf) {
-		Preconditions.checkNotNull(typeInfo);
-
-		this.typeSerializer = typeInfo.createSerializer(conf);
-	}
-
-	public TypeSerializer<T> getObjectSerializer() {
-		return typeSerializer;
-	}
-
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public StreamWindow<T> createInstance() {
-		return new StreamWindow<T>(0, 0);
-	}
-
-	@Override
-	public StreamWindow<T> copy(StreamWindow<T> from) {
-		return new StreamWindow<T>(from, typeSerializer);
-	}
-
-	@Override
-	public StreamWindow<T> copy(StreamWindow<T> from, StreamWindow<T> reuse) {
-		reuse.clear();
-		reuse.windowID = from.windowID;
-		reuse.numberOfParts = from.numberOfParts;
-		for (T element : from) {
-			reuse.add(typeSerializer.copy(element));
-		}
-		return reuse;
-	}
-
-	@Override
-	public int getLength() {
-		return -1;
-	}
-
-	@Override
-	public void serialize(StreamWindow<T> window, DataOutputView target) throws IOException {
-
-		intSerializer.serialize(window.windowID, target);
-		intSerializer.serialize(window.numberOfParts, target);
-
-		intSerializer.serialize(window.size(), target);
-
-		for (T element : window) {
-			typeSerializer.serialize(element, target);
-		}
-	}
-
-	@Override
-	public StreamWindow<T> deserialize(DataInputView source) throws IOException {
-		return deserialize(createInstance(), source);
-	}
-
-	@Override
-	public StreamWindow<T> deserialize(StreamWindow<T> reuse, DataInputView source)
-			throws IOException {
-
-		StreamWindow<T> window = reuse;
-		window.clear();
-
-		window.windowID = intSerializer.deserialize(source);
-		window.numberOfParts = intSerializer.deserialize(source);
-
-		int size = intSerializer.deserialize(source);
-
-		for (int i = 0; i < size; i++) {
-			window.add(typeSerializer.deserialize(source));
-		}
-
-		return window;
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		serialize(deserialize(source), target);
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof StreamWindowSerializer) {
-			StreamWindowSerializer<?> other = (StreamWindowSerializer<?>) obj;
-
-			return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof StreamWindowSerializer;
-	}
-
-	@Override
-	public int hashCode() {
-		return typeSerializer.hashCode();
-	}
-
-	@Override
-	public TypeSerializer<StreamWindow<T>> duplicate() {
-		return this;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java
deleted file mode 100644
index 2c0a999..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUStreamRecord<?>WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing;
-
-import com.google.common.base.Preconditions;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-public class StreamWindowTypeInfo<T> extends TypeInformation<StreamWindow<T>> {
-
-	private static final long serialVersionUID = 1L;
-
-	final TypeInformation<T> innerType;
-
-	public StreamWindowTypeInfo(TypeInformation<T> innerType) {
-		this.innerType = Preconditions.checkNotNull(innerType);
-	}
-
-	public TypeInformation<T> getInnerType() {
-		return innerType;
-	}
-
-	@Override
-	public boolean isBasicType() {
-		return innerType.isBasicType();
-	}
-
-	@Override
-	public boolean isTupleType() {
-		return innerType.isTupleType();
-	}
-
-	@Override
-	public int getArity() {
-		return innerType.getArity();
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public Class<StreamWindow<T>> getTypeClass() {
-		return (Class<StreamWindow<T>>)(Object)StreamWindow.class;
-	}
-
-	@Override
-	public boolean isKeyType() {
-		return innerType.isKeyType();
-	}
-
-	@Override
-	public TypeSerializer<StreamWindow<T>> createSerializer(ExecutionConfig conf) {
-		return new StreamWindowSerializer<T>(innerType, conf);
-	}
-
-	@Override
-	public String toString() {
-		return getClass().getSimpleName() + "<" + innerType + ">";
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof StreamWindowTypeInfo) {
-			@SuppressWarnings("unchecked")
-			StreamWindowTypeInfo<T> streamWindowTypeInfo = (StreamWindowTypeInfo<T>) obj;
-
-			return streamWindowTypeInfo.canEqual(this) &&
-				innerType.equals(streamWindowTypeInfo.innerType);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return innerType.hashCode();
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof StreamWindowTypeInfo;
-	}
-
-	@Override
-	public int getTotalFields() {
-		return innerType.getTotalFields();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowEvent.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowEvent.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowEvent.java
deleted file mode 100644
index 359dfb3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowEvent.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-
-/**
- * Type representing events sent to the window buffer. The first field should
- * contain the window element the second field encodes triggers and evictions if
- * the second field is greater than 0 it represents an eviction if it equals -1
- * it represents a trigger.
- */
-public class WindowEvent<T> extends Tuple2<T, Integer> {
-	private static final long serialVersionUID = 1L;
-
-	public boolean isElement() {
-		return f1 == 0;
-	}
-
-	public boolean isEviction() {
-		return f1 > 0;
-	}
-
-	public boolean isTrigger() {
-		return f1 == -1;
-	}
-
-	public Integer getEviction() {
-		return f1;
-	}
-
-	public T getElement() {
-		return f0;
-	}
-
-	public WindowEvent<T> setElement(T element) {
-		f0 = element;
-		f1 = 0;
-		return this;
-	}
-
-	public WindowEvent<T> setTrigger() {
-		f1 = -1;
-		return this;
-	}
-
-	public WindowEvent<T> setEviction(Integer n) {
-		if (n > 0) {
-			f1 = n;
-			return this;
-		} else {
-			throw new RuntimeException("Must evict at least 1");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowUtils.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowUtils.java
deleted file mode 100644
index a899b74..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/WindowUtils.java
+++ /dev/null
@@ -1,203 +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.flink.streaming.api.windowing;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.api.windowing.policy.CountEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.CountTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.KeepAllEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TimeEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TumblingEvictionPolicy;
-
-/**
- * Utility class that contains helper methods to work with stream windowing.
- */
-public class WindowUtils {
-
-	public enum WindowTransformation {
-		REDUCEWINDOW, MAPWINDOW, FOLDWINDOW, NONE;
-		private Function UDF;
-
-		public WindowTransformation with(Function UDF) {
-			this.UDF = UDF;
-			return this;
-		}
-
-		public Function getUDF() {
-			return UDF;
-		}
-	}
-
-	public static boolean isParallelPolicy(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction,
-			int parallelism) {
-		return ((eviction instanceof CountEvictionPolicy && (trigger instanceof CountTriggerPolicy || trigger instanceof TimeTriggerPolicy))
-				|| (eviction instanceof TumblingEvictionPolicy && trigger instanceof CountTriggerPolicy) || (WindowUtils
-				.isTimeOnly(trigger, eviction) && parallelism > 1));
-	}
-
-	public static boolean isSlidingTimePolicy(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction) {
-		if (isTimeOnly(trigger, eviction)) {
-			long slide = getSlideSize(trigger);
-			long window = getWindowSize(eviction);
-
-			return slide < window
-					&& getTimeStampWrapper(trigger).equals(getTimeStampWrapper(eviction));
-		} else {
-			return false;
-		}
-	}
-
-	public static boolean isSlidingCountPolicy(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction) {
-		if (isCountOnly(trigger, eviction)) {
-			long slide = getSlideSize(trigger);
-			long window = getWindowSize(eviction);
-
-			return slide < window
-					&& ((CountTriggerPolicy<?>) trigger).getStart() == ((CountEvictionPolicy<?>) eviction)
-							.getStart()
-					&& ((CountEvictionPolicy<?>) eviction).getDeleteOnEviction() == 1;
-		} else {
-			return false;
-		}
-	}
-
-	public static <X> TimestampWrapper<X> getTimeStampWrapper(TriggerPolicy<X> trigger) {
-		if (trigger instanceof TimeTriggerPolicy) {
-			return ((TimeTriggerPolicy<X>) trigger).getTimeStampWrapper();
-		} else {
-			throw new IllegalArgumentException(
-					"Timestamp wrapper can only be accessed for time policies");
-		}
-	}
-
-	public static <X> TimestampWrapper<X> getTimeStampWrapper(EvictionPolicy<X> eviction) {
-		if (eviction instanceof EvictionPolicy) {
-			return ((TimeEvictionPolicy<X>) eviction).getTimeStampWrapper();
-		} else {
-			throw new IllegalArgumentException(
-					"Timestamp wrapper can only be accessed for time policies");
-		}
-	}
-
-	public static long getSlideSize(TriggerPolicy<?> trigger) {
-		if (trigger instanceof TimeTriggerPolicy) {
-			return ((TimeTriggerPolicy<?>) trigger).getSlideSize();
-		} else if (trigger instanceof CountTriggerPolicy) {
-			return ((CountTriggerPolicy<?>) trigger).getSlideSize();
-		} else {
-			throw new IllegalArgumentException(
-					"Slide size can only be accessed for time or count policies");
-		}
-	}
-
-	public static long getWindowSize(EvictionPolicy<?> eviction) {
-		if (eviction instanceof TimeEvictionPolicy) {
-			return ((TimeEvictionPolicy<?>) eviction).getWindowSize();
-		} else if (eviction instanceof CountEvictionPolicy) {
-			return ((CountEvictionPolicy<?>) eviction).getWindowSize();
-		} else {
-			throw new IllegalArgumentException(
-					"Window size can only be accessed for time or count policies");
-		}
-	}
-
-	public static boolean isTumblingPolicy(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction) {
-		if (eviction instanceof TumblingEvictionPolicy || eviction instanceof KeepAllEvictionPolicy) {
-			return true;
-		} else if (isTimeOnly(trigger, eviction)) {
-			long slide = getSlideSize(trigger);
-			long window = getWindowSize(eviction);
-
-			return slide == window
-					&& getTimeStampWrapper(trigger).equals(getTimeStampWrapper(eviction));
-		} else if (isCountOnly(trigger, eviction)) {
-			long slide = getSlideSize(trigger);
-			long window = getWindowSize(eviction);
-
-			return slide == window
-					&& ((CountTriggerPolicy<?>) trigger).getStart() == ((CountEvictionPolicy<?>) eviction)
-							.getStart()
-					&& ((CountEvictionPolicy<?>) eviction).getDeleteOnEviction() == 1;
-		} else {
-			return false;
-		}
-	}
-
-	public static boolean isTimeOnly(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction) {
-		return trigger instanceof TimeTriggerPolicy
-				&& (eviction instanceof TimeEvictionPolicy || eviction instanceof KeepAllEvictionPolicy);
-	}
-
-	public static boolean isCountOnly(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction) {
-		return trigger instanceof CountTriggerPolicy && eviction instanceof CountEvictionPolicy;
-	}
-
-	public static boolean isSystemTimeTrigger(TriggerPolicy<?> trigger) {
-		return trigger instanceof TimeTriggerPolicy
-				&& ((TimeTriggerPolicy<?>) trigger).timestampWrapper.isDefaultTimestamp();
-	}
-
-	public static class WindowKey<R> implements KeySelector<StreamWindow<R>, Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer getKey(StreamWindow<R> value) throws Exception {
-			return value.windowID;
-		}
-
-	}
-
-	public static boolean isJumpingCountPolicy(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction) {
-		if (isCountOnly(trigger, eviction)) {
-			long slide = getSlideSize(trigger);
-			long window = getWindowSize(eviction);
-
-			return slide > window
-					&& ((CountTriggerPolicy<?>) trigger).getStart() == ((CountEvictionPolicy<?>) eviction)
-							.getStart()
-					&& ((CountEvictionPolicy<?>) eviction).getDeleteOnEviction() == 1;
-		} else {
-			return false;
-		}
-	}
-
-	public static boolean isJumpingTimePolicy(TriggerPolicy<?> trigger, EvictionPolicy<?> eviction) {
-		if (isTimeOnly(trigger, eviction)) {
-			long slide = getSlideSize(trigger);
-			long window = getWindowSize(eviction);
-
-			return slide > window
-					&& getTimeStampWrapper(trigger).equals(getTimeStampWrapper(eviction));
-		} else {
-			return false;
-		}
-	}
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private WindowUtils() {
-		throw new RuntimeException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
index 5004c42..5776d8d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.api.windowing.evictors;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterables;
 import org.apache.flink.streaming.api.windowing.time.AbstractTime;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -40,7 +41,7 @@ public class TimeEvictor<W extends Window> implements Evictor<Object, W> {
 	@Override
 	public int evict(Iterable<StreamRecord<Object>> elements, int size, W window) {
 		int toEvict = 0;
-		long currentTime = System.currentTimeMillis();
+		long currentTime = Iterables.getLast(elements).getTimestamp();
 		long evictCutoff = currentTime - windowSize;
 		for (StreamRecord<Object> record: elements) {
 			if (record.getTimestamp() > evictCutoff) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTuple.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTuple.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTuple.java
deleted file mode 100644
index ee878ac..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ArrayFromTuple.java
+++ /dev/null
@@ -1,74 +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.flink.streaming.api.windowing.extractor;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Converts a Tuple to an Object-Array. The field which should be included in
- * the array can selected and reordered as needed.
- */
-public class ArrayFromTuple implements Extractor<Tuple, Object[]> {
-
-	/**
-	 * Auto generated version id
-	 */
-	private static final long serialVersionUID = -6076121226427616818L;
-	int[] order = null;
-
-	/**
-	 * Using this constructor the extractor will convert the whole tuple (all
-	 * fields in the original order) to an array.
-	 */
-	public ArrayFromTuple() {
-		// noting to do
-	}
-
-	/**
-	 * Using this constructor the extractor will combine the fields as specified
-	 * in the indexes parameter in an object array.
-	 * 
-	 * @param indexes
-	 *            the field ids (enumerated from 0)
-	 */
-	public ArrayFromTuple(int... indexes) {
-		this.order = indexes;
-	}
-
-	@Override
-	public Object[] extract(Tuple in) {
-		Object[] output;
-
-		if (order == null) {
-			// copy the whole tuple
-			output = new Object[in.getArity()];
-			for (int i = 0; i < in.getArity(); i++) {
-				output[i] = in.getField(i);
-			}
-		} else {
-			// copy user specified order
-			output = new Object[order.length];
-			for (int i = 0; i < order.length; i++) {
-				output[i] = in.getField(order[i]);
-			}
-		}
-
-		return output;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtract.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtract.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtract.java
deleted file mode 100644
index a220abe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/ConcatenatedExtract.java
+++ /dev/null
@@ -1,68 +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.flink.streaming.api.windowing.extractor;
-
-/**
- * Combines two extractors which will be executed one after each other.
- *
- * @param <FROM>
- *            The input type of the first extractor.
- * @param <OVER>
- *            The output type of the first and the input type of the second
- *            extractor.
- * @param <TO>
- *            The output type of the second extractor and the output type of the
- *            over all extraction.
- */
-public class ConcatenatedExtract<FROM, OVER, TO> implements Extractor<FROM, TO> {
-
-	/**
-	 * auto-generated id
-	 */
-	private static final long serialVersionUID = -7807197760725651752L;
-
-	private Extractor<FROM, OVER> e1;
-	private Extractor<OVER, TO> e2;
-
-	/**
-	 * Combines two extractors which will be executed one after each other.
-	 * 
-	 * @param e1
-	 *            First extractor: This extractor gets applied to the input data
-	 *            first. Its output as then passed as input to the second
-	 *            extractor.
-	 * @param e2
-	 *            Second extractor: This extractor gets the output of the first
-	 *            extractor as input. Its output is then the result of the over
-	 *            all extraction.
-	 */
-	public ConcatenatedExtract(Extractor<FROM, OVER> e1, Extractor<OVER, TO> e2) {
-		this.e1 = e1;
-		this.e2 = e2;
-	}
-
-	@Override
-	public TO extract(FROM in) {
-		return e2.extract(e1.extract(in));
-	}
-
-	public <OUT> ConcatenatedExtract<FROM, TO, OUT> add(Extractor<TO, OUT> e3) {
-		return new ConcatenatedExtract<FROM, TO, OUT>(this, e3);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/Extractor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/Extractor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/Extractor.java
deleted file mode 100644
index b103ca3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/Extractor.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.extractor;
-
-import java.io.Serializable;
-
-/**
- * Extractors allow to extract/convert one type to another. They are mostly used
- * to extract some fields out of a more complex structure (Tuple/Array) to run
- * further calculation on the extraction result.
- * 
- * @param <FROM>
- *            The input data type.
- * @param <TO>
- *            The output data type.
- */
-public interface Extractor<FROM, TO> extends Serializable {
-
-	/**
-	 * Extracts/Converts the given input to an object of the output type
-	 * 
-	 * @param in
-	 *            the input data
-	 * @return the extracted/converted data
-	 */
-	public TO extract(FROM in);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArray.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArray.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArray.java
deleted file mode 100644
index 0568276..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromArray.java
+++ /dev/null
@@ -1,59 +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.flink.streaming.api.windowing.extractor;
-
-import java.lang.reflect.Array;
-
-/**
- * Extracts a single field out of an array.
- * 
- * @param <OUT>
- *            The type of the extracted field.
- */
-public class FieldFromArray<OUT> implements Extractor<Object, OUT> {
-
-	/**
-	 * Auto-gernated version id
-	 */
-	private static final long serialVersionUID = -5161386546695574359L;
-	private int fieldId = 0;
-
-	/**
-	 * Extracts the first field (id 0) from the array
-	 */
-	public FieldFromArray() {
-		// noting to do => will use default 0
-	}
-
-	/**
-	 * Extracts the field with the given id from the array.
-	 * 
-	 * @param fieldId
-	 *            The id of the field which will be extracted from the array.
-	 */
-	public FieldFromArray(int fieldId) {
-		this.fieldId = fieldId;
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public OUT extract(Object in) {
-		return (OUT) Array.get(in, fieldId);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTuple.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTuple.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTuple.java
deleted file mode 100644
index 07b38f5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldFromTuple.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.extractor;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Extracts a single field out of a tuple.
- * 
- * @param <OUT>
- *            The type of the extracted field.
- */
-public class FieldFromTuple<OUT> implements Extractor<Tuple, OUT> {
-
-	/**
-	 * Auto-gernated version id
-	 */
-	private static final long serialVersionUID = -5161386546695574359L;
-	private int fieldId = 0;
-
-	/**
-	 * Extracts the first field (id 0) from the tuple
-	 */
-	public FieldFromTuple() {
-		// noting to do => will use default 0
-	}
-
-	/**
-	 * Extracts the field with the given id from the tuple.
-	 * 
-	 * @param fieldId
-	 *            The id of the field which will be extracted from the tuple.
-	 */
-	public FieldFromTuple(int fieldId) {
-		this.fieldId = fieldId;
-	}
-
-	@Override
-	public OUT extract(Tuple in) {
-		return in.getField(fieldId);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArray.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArray.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArray.java
deleted file mode 100644
index 4e98689..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromArray.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.extractor;
-
-import java.lang.reflect.Array;
-
-/**
- * Extracts multiple fields from an array and puts them into a new array of the
- * specified type.
- *
- * @param <OUT>
- *            The type of the output array. If out is set to String, the output
- *            of the extractor will be a String[]. If it is set to String[] the
- *            output will be String[][].
- */
-public class FieldsFromArray<OUT> implements Extractor<Object, OUT[]> {
-
-	/**
-	 * Auto-generated version id
-	 */
-	private static final long serialVersionUID = 8075055384516397670L;
-	private int[] order;
-	private Class<OUT> clazz;
-
-	/**
-	 * Extracts multiple fields from an array and puts them in the given order
-	 * into a new array of the specified type.
-	 * 
-	 * @param clazz
-	 *            the Class object representing the component type of the new
-	 *            array
-	 * @param indexes
-	 *            The indexes of the fields to be extracted. Any order is
-	 *            possible, but not more than 255 fields due to limitations in
-	 *            {@link Array#newInstance(Class, int...)}.
-	 */
-	public FieldsFromArray(Class<OUT> clazz, int... indexes) {
-		this.order = indexes;
-		this.clazz = clazz;
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public OUT[] extract(Object in) {
-		OUT[] output = (OUT[]) Array.newInstance(clazz, order.length);
-		for (int i = 0; i < order.length; i++) {
-			output[i] = (OUT) Array.get(in, this.order[i]);
-		}
-		return output;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTuple.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTuple.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTuple.java
deleted file mode 100644
index 1bfc461..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/extractor/FieldsFromTuple.java
+++ /dev/null
@@ -1,53 +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.flink.streaming.api.windowing.extractor;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Extracts one or more fields of the type Double from a tuple and puts them
- * into a new double[]
- */
-public class FieldsFromTuple implements Extractor<Tuple, double[]> {
-
-	/**
-	 * auto generated version id
-	 */
-	private static final long serialVersionUID = -2554079091050273761L;
-	int[] indexes;
-
-	/**
-	 * Extracts one or more fields of the the type Double from a tuple and puts
-	 * them into a new double[] (in the specified order).
-	 * 
-	 * @param indexes
-	 *            The indexes of the fields to be extracted.
-	 */
-	public FieldsFromTuple(int... indexes) {
-		this.indexes = indexes;
-	}
-
-	@Override
-	public double[] extract(Tuple in) {
-		double[] out = new double[indexes.length];
-		for (int i = 0; i < indexes.length; i++) {
-			out[i] = (Double) in.getField(indexes[i]);
-		}
-		return out;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Count.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Count.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Count.java
deleted file mode 100644
index 3266a24..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Count.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.helper;
-
-import org.apache.flink.streaming.api.windowing.policy.CountEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.CountTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-
-/**
- * Represents a count based trigger or eviction policy. Use the
- * {@link Count#of(int)} to get an instance.
- */
-@SuppressWarnings("rawtypes")
-public class Count extends WindowingHelper {
-
-	private int count;
-	private int deleteOnEviction = 1;
-	private int startValue = CountTriggerPolicy.DEFAULT_START_VALUE;
-
-	/**
-	 * Specifies on which element a trigger or an eviction should happen (based
-	 * on the count of the elements).
-	 * 
-	 * This constructor does exactly the same as {@link Count#of(int)}.
-	 * 
-	 * @param count
-	 *            the number of elements to count before trigger/evict
-	 */
-	public Count(int count) {
-		this.count = count;
-	}
-
-	@Override
-	public EvictionPolicy<?> toEvict() {
-		return new CountEvictionPolicy(count, deleteOnEviction);
-	}
-
-	@Override
-	public TriggerPolicy<?> toTrigger() {
-		return new CountTriggerPolicy(count, startValue);
-	}
-
-	/**
-	 * Sets the number of elements deleted at each eviction (i.e when the number
-	 * elements exceeds the window size). By default the elements get deleted
-	 * one by one (deleteOnEvition = 1)
-	 * 
-	 * @param deleteOnEviction
-	 *            The number of elements deleted at each evition
-	 * @return Helper representing the count based policy
-	 * 
-	 */
-	public Count withDelete(int deleteOnEviction) {
-		this.deleteOnEviction = deleteOnEviction;
-		return this;
-	}
-
-	/**
-	 * Sets the initial value of the counter. 0 by default
-	 * 
-	 * @param startValue
-	 *            Starting value of the window counter
-	 * @return Helper representing the count based policy
-	 * 
-	 */
-	public Count startingAt(int startValue) {
-		this.startValue = startValue;
-		return this;
-	}
-
-	/**
-	 * Specifies a count based eviction (window size) or trigger policy (slide
-	 * size). For eviction 'count' defines the number of elements in each
-	 * window. For trigger 'count' defines how often do we call the user
-	 * function in terms of number of elements received.
-	 * 
-	 * @param count
-	 *            the number of elements to count before trigger/evict
-	 * @return Helper representing the count based policy
-	 */
-	public static Count of(int count) {
-		return new Count(count);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Delta.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Delta.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Delta.java
deleted file mode 100644
index 31063ab..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Delta.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.helper;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.windowing.policy.DeltaPolicy;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-
-/**
- * This helper represents a trigger or eviction policy based on a
- * {@link DeltaFunction}.
- * 
- * @param <DATA>
- *            the data type handled by the delta function represented by this
- *            helper.
- */
-public class Delta<DATA> extends WindowingHelper<DATA> {
-
-	private DeltaFunction<DATA> deltaFunction;
-	private DATA initVal;
-	private double threshold;
-	private TypeSerializer<DATA> typeSerializer;
-
-	/**
-	 * Creates a delta helper representing a delta count or eviction policy
-	 * @param deltaFunction
-	 *				The delta function which should be used to calculate the delta
-	 *				points.
-	 * @param initVal
-	 *				The initial value which will be used to calculate the first
-	 *				delta.
-	 * @param threshold
-	 * 				The threshold used by the delta function.
-	 */
-	public Delta(DeltaFunction<DATA> deltaFunction, DATA initVal, double threshold) {
-		this.deltaFunction = deltaFunction;
-		this.initVal = initVal;
-		this.threshold = threshold;
-	}
-
-	@Override
-	public EvictionPolicy<DATA> toEvict() {
-		instantiateTypeSerializer();
-		return new DeltaPolicy<DATA>(deltaFunction, initVal, threshold, typeSerializer);
-	}
-
-	@Override
-	public TriggerPolicy<DATA> toTrigger() {
-		instantiateTypeSerializer();
-		return new DeltaPolicy<DATA>(deltaFunction, initVal, threshold, typeSerializer);
-	}
-
-	/**
-	 * Creates a delta helper representing a delta trigger or eviction policy.
-	 * </br></br> This policy calculates a delta between the data point which
-	 * triggered last and the currently arrived data point. It triggers if the
-	 * delta is higher than a specified threshold. </br></br> In case it gets
-	 * used for eviction, this policy starts from the first element of the
-	 * buffer and removes all elements from the buffer which have a higher delta
-	 * then the threshold. As soon as there is an element with a lower delta,
-	 * the eviction stops.
-	 *
-	 * @param deltaFunction
-	 *				The delta function which should be used to calculate the delta
-	 *				points.
-	 * @param initVal
-	 *				The initial value which will be used to calculate the first
-	 *				delta.
-	 * @param threshold
-	 * 				The threshold used by the delta function.
-	 * @return Helper representing a delta trigger or eviction policy
-	 */
-	public static <DATA> Delta<DATA> of(double threshold, DeltaFunction<DATA> deltaFunction,
-			DATA initVal) {
-		return new Delta<DATA>(deltaFunction, initVal, threshold);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void instantiateTypeSerializer(){
-		if (executionConfig == null){
-			throw new UnsupportedOperationException("ExecutionConfig has to be set to instantiate TypeSerializer.");
-		}
-		TypeInformation typeInformation = TypeExtractor.getForObject(initVal);
-		typeSerializer = typeInformation.createSerializer(executionConfig);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/FullStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/FullStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/FullStream.java
deleted file mode 100644
index 7773d9a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/FullStream.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.helper;
-
-import java.io.Serializable;
-
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.KeepAllEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-
-/**
- * Window that represents the full stream history. Can be used only as eviction
- * policy and only with operations that support pre-aggregator such as reduce or
- * aggregations.
- */
-public class FullStream<DATA> extends WindowingHelper<DATA> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private FullStream() {
-	}
-
-	@Override
-	public EvictionPolicy<DATA> toEvict() {
-		return new KeepAllEvictionPolicy<DATA>();
-	}
-
-	@Override
-	public TriggerPolicy<DATA> toTrigger() {
-		throw new RuntimeException(
-				"Full stream policy can be only used as eviction. Use .every(..) after the window call.");
-	}
-
-	/**
-	 * Returns a helper representing an eviction that keeps all previous record
-	 * history.
-	 */
-	public static <R> FullStream<R> window() {
-		return new FullStream<R>();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/SystemTimestamp.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/SystemTimestamp.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/SystemTimestamp.java
deleted file mode 100644
index 8581ac5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/SystemTimestamp.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.helper;
-
-/**
- * {@link Timestamp} implementation to be used when system time is needed to
- * determine windows
- */
-public class SystemTimestamp<T> implements Timestamp<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public long getTimestamp(T value) {
-		return System.currentTimeMillis();
-	}
-
-	public static <R> TimestampWrapper<R> getWrapper() {
-		return new TimestampWrapper<R>(new SystemTimestamp<R>(), System.currentTimeMillis());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Time.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Time.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Time.java
deleted file mode 100644
index 022f975..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Time.java
+++ /dev/null
@@ -1,153 +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.flink.streaming.api.windowing.helper;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TimeEvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TimeTriggerPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-
-/**
- * This helper represents a time based count or eviction policy. By default the
- * time is measured with {@link System#currentTimeMillis()} in
- * {@link SystemTimestamp}.
- * 
- * @param <DATA>
- *            The data type which is handled by the time stamp used in the
- *            policy represented by this helper
- */
-public class Time<DATA> extends WindowingHelper<DATA> {
-
-	protected long length;
-	protected TimeUnit granularity;
-	protected TimestampWrapper<DATA> timestampWrapper;
-	protected long delay;
-
-	/**
-	 * Creates a helper representing a trigger which triggers every given
-	 * length or an eviction which evicts all elements older than length.
-	 * 
-	 * @param length
-	 *            The number of time units
-	 * @param timeUnit
-	 *            The unit of time such as minute oder millisecond. Note that
-	 *            the smallest possible granularity is milliseconds. Any smaller
-	 *            time unit might cause an error at runtime due to conversion
-	 *            problems.
-	 * @param timestamp
-	 *            The user defined timestamp that will be used to extract time
-	 *            information from the incoming elements
-	 * @param startTime
-	 *            The startTime of the stream for computing the first window
-	 */
-	private Time(long length, TimeUnit timeUnit, Timestamp<DATA> timestamp, long startTime) {
-		this(length, timeUnit, new TimestampWrapper<DATA>(timestamp, startTime));
-	}
-
-	/**
-	 * Creates a helper representing a trigger which triggers every given
-	 * length or an eviction which evicts all elements older than length.
-	 * 
-	 * @param length
-	 *            The number of time units
-	 * @param timeUnit
-	 *            The unit of time such as minute oder millisecond. Note that
-	 *            the smallest possible granularity is milliseconds. Any smaller
-	 *            time unit might cause an error at runtime due to conversion
-	 *            problems.
-	 * @param timestampWrapper
-	 *            The user defined {@link TimestampWrapper} that will be used to
-	 *            extract time information from the incoming elements
-	 */
-	private Time(long length, TimeUnit timeUnit, TimestampWrapper<DATA> timestampWrapper) {
-		this.length = length;
-		this.granularity = timeUnit;
-		this.timestampWrapper = timestampWrapper;
-	}
-
-	@Override
-	public EvictionPolicy<DATA> toEvict() {
-		return new TimeEvictionPolicy<DATA>(granularityInMillis(), timestampWrapper);
-	}
-
-	@Override
-	public TriggerPolicy<DATA> toTrigger() {
-		return new TimeTriggerPolicy<DATA>(granularityInMillis(), timestampWrapper);
-	}
-
-	/**
-	 * Creates a helper representing a time trigger which triggers every given
-	 * length (slide size) or a time eviction which evicts all elements older
-	 * than length (window size) using System time.
-	 * 
-	 * @param length
-	 *            The number of time units
-	 * @param timeUnit
-	 *            The unit of time such as minute oder millisecond. Note that
-	 *            the smallest possible granularity is milliseconds. Any smaller
-	 *            time unit might cause an error at runtime due to conversion
-	 *            problems.
-	 * @return Helper representing the time based trigger and eviction policy
-	 */
-	@SuppressWarnings("unchecked")
-	public static <DATA> Time<DATA> of(long length, TimeUnit timeUnit) {
-		return new Time<DATA>(length, timeUnit,
-				(TimestampWrapper<DATA>) SystemTimestamp.getWrapper());
-	}
-
-	/**
-	 * Creates a helper representing a time trigger which triggers every given
-	 * length (slide size) or a time eviction which evicts all elements older
-	 * than length (window size) using a user defined timestamp extractor.
-	 * 
-	 * @param length
-	 *            The number of time units
-	 * @param timestamp
-	 *            The user defined timestamp that will be used to extract time
-	 *            information from the incoming elements
-	 * @param startTime
-	 *            The startTime used to compute the first window
-	 * @return Helper representing the time based trigger and eviction policy
-	 */
-	public static <DATA> Time<DATA> of(long length, Timestamp<DATA> timestamp, long startTime) {
-		return new Time<DATA>(length, null, timestamp, startTime);
-	}
-
-	/**
-	 * Creates a helper representing a time trigger which triggers every given
-	 * length (slide size) or a time eviction which evicts all elements older
-	 * than length (window size) using a user defined timestamp extractor. By
-	 * default the start time is set to 0.
-	 * 
-	 * @param length
-	 *            The number of time units
-	 * @param timestamp
-	 *            The user defined timestamp that will be used to extract time
-	 *            information from the incoming elements
-	 * @return Helper representing the time based trigger and eviction policy
-	 */
-	public static <DATA> Time<DATA> of(long length, Timestamp<DATA> timestamp) {
-		return of(length, timestamp, 0);
-	}
-	
-	protected long granularityInMillis() {
-		return granularity == null ? length : granularity.toMillis(length);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Timestamp.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Timestamp.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Timestamp.java
deleted file mode 100644
index fea6020..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/Timestamp.java
+++ /dev/null
@@ -1,39 +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.flink.streaming.api.windowing.helper;
-
-import java.io.Serializable;
-
-/**
- * Interface for getting a timestamp from a custom value. Used in window
- * reduces. In order to work properly, the timestamps must be non-decreasing.
- *
- * @param <T>
- *            Type of the value to create the timestamp from.
- */
-public interface Timestamp<T> extends Serializable {
-
-	/**
-	 * Values
-	 * 
-	 * @param value
-	 *            The value to create the timestamp from
-	 * @return The timestamp
-	 */
-	public long getTimestamp(T value);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/TimestampWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/TimestampWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/TimestampWrapper.java
deleted file mode 100644
index c2ec7c2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/TimestampWrapper.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.flink.streaming.api.windowing.helper;
-
-import java.io.Serializable;
-
-public class TimestampWrapper<T> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-	private long startTime;
-	private Timestamp<T> timestamp;
-
-	public TimestampWrapper(Timestamp<T> timeStamp, long startTime) {
-		this.timestamp = timeStamp;
-		this.startTime = startTime;
-	}
-
-	public long getTimestamp(T in) {
-		return timestamp.getTimestamp(in);
-	}
-
-	public long getStartTime() {
-		return startTime;
-	}
-
-	public boolean isDefaultTimestamp() {
-		return timestamp instanceof SystemTimestamp;
-	}
-
-	@Override
-	public boolean equals(Object other) {
-		if (other == null || !(other instanceof TimestampWrapper)) {
-			return false;
-		} else {
-			try {
-				@SuppressWarnings("unchecked")
-				TimestampWrapper<T> otherTSW = (TimestampWrapper<T>) other;
-				if (timestamp instanceof SystemTimestamp
-						&& otherTSW.timestamp instanceof SystemTimestamp) {
-					return true;
-				} else {
-					return startTime == otherTSW.startTime
-							&& timestamp.getClass() == otherTSW.timestamp.getClass();
-				}
-			} catch (ClassCastException e) {
-				return false;
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/WindowingHelper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/WindowingHelper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/WindowingHelper.java
deleted file mode 100644
index 17e142a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/helper/WindowingHelper.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.flink.streaming.api.windowing.helper;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
-import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
-
-/**
- * A helper representing a count or eviction policy. Such helper classes are
- * used to provide a nice and well readable API.
- * 
- * @param <DATA>
- *            the type of input data handled by this helper
- * @see Count
- * @see Time
- * @see Delta
- */
-public abstract class WindowingHelper<DATA> {
-
-	/**
-	 * Provides information for initial value serialization
-	 * in {@link Delta}, unused in other subclasses.
-	 */
-	protected ExecutionConfig executionConfig;
-
-	/**
-	 * Method for encapsulating the {@link EvictionPolicy}.
-	 * @return the eviction policy
-	 */
-	public abstract EvictionPolicy<DATA> toEvict();
-
-	/**
-	 * Method for encapsulating the {@link TriggerPolicy}.
-	 * @return the trigger policy
-	 */
-	public abstract TriggerPolicy<DATA> toTrigger();
-
-	/**
-	 * Setter for the {@link ExecutionConfig} field.
-	 * @param executionConfig Desired value
-	 */
-	public final void setExecutionConfig(ExecutionConfig executionConfig){
-		this.executionConfig = executionConfig;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveCloneableEvictionPolicyWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveCloneableEvictionPolicyWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveCloneableEvictionPolicyWrapper.java
deleted file mode 100644
index 29ba9eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveCloneableEvictionPolicyWrapper.java
+++ /dev/null
@@ -1,62 +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.flink.streaming.api.windowing.policy;
-
-/**
- * The {@link ActiveEvictionPolicy} wraps around a non active
- * {@link EvictionPolicy}. It forwards all calls to
- * {@link ActiveEvictionPolicy#notifyEvictionWithFakeElement(Object, int)} to
- * {@link EvictionPolicy#notifyEviction(Object, boolean, int)} while the
- * triggered parameter will be set to true.
- * 
- * This class additionally implements the clone method and can wrap around
- * {@link CloneableEvictionPolicy} to make it active.
- * 
- * @param <DATA>
- *            The data type handled by this policy
- */
-public class ActiveCloneableEvictionPolicyWrapper<DATA> extends ActiveEvictionPolicyWrapper<DATA>
-		implements CloneableEvictionPolicy<DATA> {
-
-	/**
-	 * Auto generated version ID
-	 */
-	private static final long serialVersionUID = 1520261575300622769L;
-	CloneableEvictionPolicy<DATA> nestedPolicy;
-
-	/**
-	 * Creates a wrapper which activates the eviction policy which is wrapped
-	 * in. This means that the nested policy will get called on fake elements as
-	 * well as on real elements.
-	 * 
-	 * This specialized version of the {@link ActiveEvictionPolicyWrapper} works
-	 * with {@link CloneableEvictionPolicy} and is thereby cloneable as well.
-	 * 
-	 * @param nestedPolicy
-	 *            The policy which should be activated/wrapped in.
-	 */
-	public ActiveCloneableEvictionPolicyWrapper(CloneableEvictionPolicy<DATA> nestedPolicy) {
-		super(nestedPolicy);
-		this.nestedPolicy = nestedPolicy;
-	}
-
-	@Override
-	public ActiveCloneableEvictionPolicyWrapper<DATA> clone() {
-		return new ActiveCloneableEvictionPolicyWrapper<DATA>(nestedPolicy.clone());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicy.java
deleted file mode 100644
index fe172bc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicy.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-/**
- * This interface is used for active eviction policies. beside the functionality
- * inherited from {@link EvictionPolicy} it provides a method which gets called
- * to notify on fake elements.
- * 
- * In case an eviction policy implements this interface instead of the
- * {@link EvictionPolicy} interface, not only the real but also fake data points
- * will cause a notification of the eviction.
- * 
- * Fake data points are mostly used in windowing based on time to trigger and
- * evict even if no element arrives at all during a windows duration.
- */
-public interface ActiveEvictionPolicy<DATA> extends EvictionPolicy<DATA> {
-
-	/**
-	 * Proves if and how many elements should be deleted from the element
-	 * buffer. The eviction takes place after the trigger and after the call to
-	 * the UDF. This method is only called with fake elements.
-	 * 
-	 * Note: Fake elements are always considered as triggered. Therefore this
-	 * method does not have a triggered parameter.
-	 * 
-	 * @param datapoint
-	 *            the current fake data point
-	 * @param bufferSize
-	 *            the current size of the buffer (only real elements are
-	 *            counted)
-	 * @return the number of elements to delete from the buffer (only real
-	 *         elements are counted)
-	 */
-	public int notifyEvictionWithFakeElement(Object datapoint, int bufferSize);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicyWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicyWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicyWrapper.java
deleted file mode 100644
index b3b6935..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveEvictionPolicyWrapper.java
+++ /dev/null
@@ -1,64 +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.flink.streaming.api.windowing.policy;
-
-/**
- * This {@link ActiveEvictionPolicy} wraps around a non active
- * {@link EvictionPolicy}. It forwards all calls to
- * {@link ActiveEvictionPolicy#notifyEvictionWithFakeElement(Object, int)} to
- * {@link EvictionPolicy#notifyEviction(Object, boolean, int)} while the
- * triggered parameter will be set to true.
- * 
- * @param <DATA>
- *            The data type handled by this policy
- */
-public class ActiveEvictionPolicyWrapper<DATA> implements ActiveEvictionPolicy<DATA> {
-
-	/**
-	 * Auto generated version ID
-	 */
-	private static final long serialVersionUID = -7656558669799505882L;
-	private EvictionPolicy<DATA> nestedPolicy;
-
-	/**
-	 * Creates a wrapper which activates the eviction policy which is wrapped
-	 * in. This means that the nested policy will get called on fake elements as
-	 * well as on real elements.
-	 * 
-	 * @param nestedPolicy
-	 *            The policy which should be activated/wrapped in.
-	 */
-	public ActiveEvictionPolicyWrapper(EvictionPolicy<DATA> nestedPolicy) {
-		if (nestedPolicy == null) {
-			throw new RuntimeException("The nested policy must not be null.");
-		}
-		this.nestedPolicy = nestedPolicy;
-	}
-
-	@Override
-	public int notifyEviction(DATA datapoint, boolean triggered, int bufferSize) {
-		return nestedPolicy.notifyEviction(datapoint, triggered, bufferSize);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public int notifyEvictionWithFakeElement(Object datapoint, int bufferSize) {
-		return nestedPolicy.notifyEviction((DATA) datapoint, true, bufferSize);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerCallback.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerCallback.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerCallback.java
deleted file mode 100644
index c44be37..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerCallback.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-/**
- * In case an {@link ActiveTriggerPolicy} is used, it can implement own
- * {@link Runnable} classes. Such {@link Runnable} classes will be executed as
- * an own thread and can submit fake elements, to the element buffer at any
- * time.
- * 
- * The factory method for runnables of the {@link ActiveTriggerPolicy} gets an
- * instance of this interface as parameter. The describes adding of elements can
- * be done by the runnable using the methods provided in this interface.
- * 
- */
-public interface ActiveTriggerCallback {
-
-	/**
-	 * Submits a new fake data point to the element buffer. Such a fake element
-	 * might be used to trigger at any time, but will never be included in the
-	 * result of the reduce function. The submission of a fake element causes
-	 * notifications only at the {@link ActiveTriggerPolicy} and
-	 * {@link ActiveEvictionPolicy} implementations.
-	 * 
-	 * @param datapoint
-	 *            the fake data point to be added
-	 */
-	public void sendFakeElement(Object datapoint);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerPolicy.java
deleted file mode 100644
index b645c0f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/ActiveTriggerPolicy.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-
-/**
- * This interface extends the {@link TriggerPolicy} interface with functionality
- * for active triggers. Active triggers can act in two ways:
- * 
- * 1) Whenever an element arrives at the operator, the
- * {@link ActiveTriggerPolicy#preNotifyTrigger(Object)} method gets called
- * first. It can return zero ore more fake data points which will be added
- * before the currently arrived real element gets processed. This allows to
- * handle empty windows in time based windowing with an user defined
- * {@link Timestamp}. Triggers are not called on fake datapoint. A fake
- * datapoint is always considered as triggered.
- * 
- * 2) An active trigger has a factory method for a runnable. This factory method
- * gets called at the start up of the operator. The returned runnable will be
- * executed in its own thread and can submit fake elements at any time through an
- * {@link ActiveTriggerCallback}. This allows to have time based triggers based
- * on any system internal time measure. Triggers are not called on fake
- * datapoint. A fake datapoints is always considered as triggered.
- * 
- * @param <DATA>
- *            The data type which can be handled by this policy
- */
-public interface ActiveTriggerPolicy<DATA> extends TriggerPolicy<DATA> {
-
-	/**
-	 * Whenever an element arrives at the operator, the
-	 * {@link ActiveTriggerPolicy#preNotifyTrigger(Object)} method gets called
-	 * first. It can return zero ore more fake data points which will be added
-	 * before the the currently arrived real element gets processed. This allows
-	 * to handle empty windows in time based windowing with an user defined
-	 * {@link Timestamp}. Triggers are not called on fake datapoints. A fake
-	 * datapoint is always considered as triggered.
-	 * 
-	 * @param datapoint
-	 *            the data point which arrived at the operator
-	 * @return zero ore more fake data points which will be added before the the
-	 *         currently arrived real element gets processed.
-	 */
-	public Object[] preNotifyTrigger(DATA datapoint);
-
-	/**
-	 * This is the factory method for a runnable. This factory method gets
-	 * called at the start up of the operator. The returned runnable will be
-	 * executed in its own thread and can submit fake elements at any time through
-	 * an {@link ActiveTriggerCallback}. This allows to have time based triggers
-	 * based on any system internal time measure. Triggers are not called on
-	 * fake datapoints. A fake datapoint is always considered as triggered.
-	 * 
-	 * @param callback
-	 *            A callback object which allows to add fake elements from
-	 *            within the returned {@link Runnable}.
-	 * @return The runnable implementation or null in case there is no. In case
-	 *         an {@link ActiveTriggerPolicy} is used, it can implement own
-	 *         {@link Runnable} classes. Such {@link Runnable} classes will be
-	 *         executed as an own thread and can submit fake elements, to the
-	 *         element buffer at any time.
-	 */
-	public Runnable createActiveTriggerRunnable(ActiveTriggerCallback callback);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CentralActiveTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CentralActiveTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CentralActiveTrigger.java
deleted file mode 100644
index 308f152..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/policy/CentralActiveTrigger.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.policy;
-
-/**
- * Interface for defining grouped windowing policies which can interact with
- * other groups to trigger on the latest available information globally
- * available to all groups.</p> At predefined time intervals the discretizers
- * takes the last globally seen element, and notifies all groups (but the one
- * that already have seen the object). This allows to trigger before an element
- * comes from the next window for a specific group. This pattern can be
- * used for instance in time policies to regularly broadcast the current time to
- * all groups.
- */
-public interface CentralActiveTrigger<DATA> extends CloneableTriggerPolicy<DATA> {
-
-	/**
-	 * This method is called to broadcast information about the last globally
-	 * seen data point to all triggers. The number of elements returned in the
-	 * array will determine the number of triggers at that point, while the
-	 * elements themselves are used only for active eviction.
-	 * 
-	 * @param datapoint
-	 *            The last globally seen data
-	 * @return An object of fake elements. If returned null or empty list, no
-	 *         triggers will occur.
-	 */
-	public Object[] notifyOnLastGlobalElement(DATA datapoint);
-
-}


[02/10] flink git commit: [FLINK-2780] Remove Old Windowing Logic and API

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java
deleted file mode 100644
index ce312d3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/JumpingTimePreReducerTest.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class JumpingTimePreReducerTest {
-
-	TypeSerializer<Integer> serializer = TypeExtractor.getForObject(1).createSerializer(null);
-
-	ReduceFunction<Integer> reducer = new SumReducer();
-
-	@Test
-	public void testEmitWindow() throws Exception {
-
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-		List<StreamWindow<Integer>> collected = collector.getCollected();
-
-		WindowBuffer<Integer> wb = new JumpingTimePreReducer<Integer>(
-				reducer, serializer, 3, 2, new TimestampWrapper<Integer>(new Timestamp<Integer>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public long getTimestamp(Integer value) {
-				return value;
-			}
-		}, 1));
-
-		wb.store(1);
-		wb.store(2);
-		wb.store(3);
-		wb.evict(1);
-		wb.emitWindow(collector);
-
-		assertEquals(1, collected.size());
-		assertEquals(StreamWindow.fromElements(5),
-				collected.get(0));
-
-		wb.store(4);
-		wb.store(5);
-
-		// Nothing should happen here
-		wb.evict(2);
-
-		wb.store(6);
-
-		wb.emitWindow(collector);
-		wb.evict(2);
-		wb.emitWindow(collector);
-		wb.store(12);
-		wb.emitWindow(collector);
-
-		assertEquals(3, collected.size());
-		assertEquals(StreamWindow.fromElements(11),
-				collected.get(1));
-		assertEquals(StreamWindow.fromElements(12),
-				collected.get(2));
-	}
-
-	private static class SumReducer implements ReduceFunction<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1 + value2;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java
deleted file mode 100644
index 7f58527..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountGroupedPreReducerTest.java
+++ /dev/null
@@ -1,235 +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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.operators.windowing.WindowingITCase;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class SlidingCountGroupedPreReducerTest {
-
-	TypeSerializer<Integer> serializer = TypeExtractor.getForObject(1).createSerializer(null);
-
-	ReduceFunction<Integer> reducer = new SumReducer();
-
-	KeySelector<Integer, ?> key = new WindowingITCase.ModKey(2);
-
-	@Test
-	public void testPreReduce1() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountGroupedPreReducer<Integer> preReducer = new SlidingCountGroupedPreReducer<Integer>(
-				reducer, serializer, key, 3, 2, 0);
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(9);
-		preReducer.store(10);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1, 2));
-		expected.add(StreamWindow.fromElements(3, 6));
-		expected.add(StreamWindow.fromElements(5, 10));
-		expected.add(StreamWindow.fromElements(7, 14));
-		expected.add(StreamWindow.fromElements(9, 18));
-		expected.add(StreamWindow.fromElements(11, 22));
-
-		checkResults(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce2() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountGroupedPreReducer<Integer> preReducer = new SlidingCountGroupedPreReducer<Integer>(
-				reducer, serializer, key, 5, 2, 0);
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.emitWindow(collector);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(9);
-		preReducer.store(10);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1, 2));
-		expected.add(StreamWindow.fromElements(4, 6));
-		expected.add(StreamWindow.fromElements(12, 8));
-		expected.add(StreamWindow.fromElements(18, 12));
-		expected.add(StreamWindow.fromElements(24, 16));
-		expected.add(StreamWindow.fromElements(30, 20));
-
-		checkResults(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce3() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountGroupedPreReducer<Integer> preReducer = new SlidingCountGroupedPreReducer<Integer>(
-				reducer, serializer, key, 6, 3, 0);
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.store(3);
-		preReducer.emitWindow(collector);
-		preReducer.store(4);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.store(9);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(10);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(2, 4));
-		expected.add(StreamWindow.fromElements(9, 12));
-		expected.add(StreamWindow.fromElements(21, 18));
-		expected.add(StreamWindow.fromElements(30, 27));
-
-		checkResults(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce4() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountGroupedPreReducer<Integer> preReducer = new SlidingCountGroupedPreReducer<Integer>(
-				reducer, serializer, key, 5, 1, 2);
-
-		preReducer.store(1);
-		preReducer.evict(1);
-		preReducer.store(1);
-		preReducer.evict(1);
-		preReducer.store(1);
-		preReducer.emitWindow(collector);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.emitWindow(collector);
-		preReducer.store(4);
-		preReducer.emitWindow(collector);
-		preReducer.store(5);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.store(7);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1));
-		expected.add(StreamWindow.fromElements(1, 2));
-		expected.add(StreamWindow.fromElements(4, 2));
-		expected.add(StreamWindow.fromElements(4, 6));
-		expected.add(StreamWindow.fromElements(9, 6));
-		expected.add(StreamWindow.fromElements(8, 12));
-		expected.add(StreamWindow.fromElements(15, 10));
-		expected.add(StreamWindow.fromElements(12, 18));
-
-		checkResults(expected, collector.getCollected());
-	}
-
-	private static class SumReducer implements ReduceFunction<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1 + value2;
-		}
-
-	}
-
-
-	protected static void checkResults(List<StreamWindow<Integer>> expected,
-			List<StreamWindow<Integer>> actual) {
-
-		for (StreamWindow<Integer> sw : expected) {
-			Collections.sort(sw);
-		}
-
-		for (StreamWindow<Integer> sw : actual) {
-			Collections.sort(sw);
-		}
-
-		assertEquals(expected, actual);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java
deleted file mode 100644
index 156b875..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingCountPreReducerTest.java
+++ /dev/null
@@ -1,216 +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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class SlidingCountPreReducerTest {
-
-	TypeSerializer<Integer> serializer = TypeExtractor.getForObject(1).createSerializer(null);
-
-	ReduceFunction<Integer> reducer = new SumReducer();
-
-	@Test
-	public void testPreReduce1() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountPreReducer<Integer> preReducer = new SlidingCountPreReducer<Integer>(reducer,
-				serializer, 3, 2, 0);
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(9);
-		preReducer.store(10);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(3));
-		expected.add(StreamWindow.fromElements(9));
-		expected.add(StreamWindow.fromElements(15));
-		expected.add(StreamWindow.fromElements(21));
-		expected.add(StreamWindow.fromElements(27));
-		expected.add(StreamWindow.fromElements(33));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce2() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountPreReducer<Integer> preReducer = new SlidingCountPreReducer<Integer>(reducer,
-				serializer, 5, 2, 0);
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.emitWindow(collector);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(9);
-		preReducer.store(10);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(3));
-		expected.add(StreamWindow.fromElements(10));
-		expected.add(StreamWindow.fromElements(20));
-		expected.add(StreamWindow.fromElements(30));
-		expected.add(StreamWindow.fromElements(40));
-		expected.add(StreamWindow.fromElements(50));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce3() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountPreReducer<Integer> preReducer = new SlidingCountPreReducer<Integer>(reducer,
-				serializer, 6, 3, 0);
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.store(3);
-		preReducer.emitWindow(collector);
-		preReducer.store(4);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.store(9);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(10);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(6));
-		expected.add(StreamWindow.fromElements(21));
-		expected.add(StreamWindow.fromElements(39));
-		expected.add(StreamWindow.fromElements(57));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce4() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingCountPreReducer<Integer> preReducer = new SlidingCountPreReducer<Integer>(reducer,
-				serializer, 5, 1, 2);
-
-		preReducer.store(1);
-		preReducer.evict(1);
-		preReducer.store(1);
-		preReducer.evict(1);
-		preReducer.store(1);
-		preReducer.emitWindow(collector);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.emitWindow(collector);
-		preReducer.store(4);
-		preReducer.emitWindow(collector);
-		preReducer.store(5);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.store(7);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1));
-		expected.add(StreamWindow.fromElements(3));
-		expected.add(StreamWindow.fromElements(6));
-		expected.add(StreamWindow.fromElements(10));
-		expected.add(StreamWindow.fromElements(15));
-		expected.add(StreamWindow.fromElements(20));
-		expected.add(StreamWindow.fromElements(25));
-		expected.add(StreamWindow.fromElements(30));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	private static class SumReducer implements ReduceFunction<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1 + value2;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java
deleted file mode 100644
index 68bceda..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimeGroupedPreReducerTest.java
+++ /dev/null
@@ -1,387 +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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.streaming.api.operators.windowing.WindowingITCase;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class SlidingTimeGroupedPreReducerTest {
-
-	TypeSerializer<Integer> serializer = TypeExtractor.getForObject(1).createSerializer(null);
-	TypeInformation<Tuple2<Integer,Integer>> tupleType = TypeInfoParser.parse("Tuple2<Integer,Integer>");
-
-
-	ReduceFunction<Integer> reducer = new SumReducer();
-	ReduceFunction<Tuple2<Integer, Integer>> tupleReducer = new TupleSumReducer();
-
-
-	KeySelector<Integer, ?> key = new WindowingITCase.ModKey(2);
-	KeySelector<Tuple2<Integer, Integer>, ?> tupleKey = new TupleModKey(2);
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testPreReduce1() throws Exception {
-		// This ensures that the buffer is properly cleared after a burst of elements by
-		// replaying the same sequence of elements with a later timestamp and expecting the same
-		// result.
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-
-		SlidingTimeGroupedPreReducer<Tuple2<Integer, Integer>> preReducer = new SlidingTimeGroupedPreReducer<Tuple2<Integer, Integer>>(tupleReducer,
-				tupleType.createSerializer(new ExecutionConfig()), tupleKey, 3, 2, new TimestampWrapper<Tuple2<Integer, Integer>>(new Timestamp<Tuple2<Integer, Integer>>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public long getTimestamp(Tuple2<Integer, Integer> value) {
-				return value.f0;
-			}
-		}, 1));
-
-		int timeOffset = 0;
-
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 1, 1));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 2, 2));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 3, 3));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 4, 4));
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 5, 5));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 6, 6));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 7, 7));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 8, 8));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 9, 9));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 10, 10));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 11, 11));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 12, 12));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 13, 13));
-
-		// ensure that everything is cleared out
-		preReducer.evict(100);
-
-
-		timeOffset = 25; // a little while later...
-
-		// Repeat the same sequence, this should produce the same result
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 1, 1));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 2, 2));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 3, 3));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 4, 4));
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 5, 5));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 6, 6));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 7, 7));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 8, 8));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 9, 9));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 10, 10));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 11, 11));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 12, 12));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 13, 13));
-
-		List<StreamWindow<Tuple2<Integer, Integer>>> expected = new ArrayList<StreamWindow<Tuple2<Integer, Integer>>>();
-		timeOffset = 0; // rewind ...
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 2, 2),
-				new Tuple2<Integer, Integer>(timeOffset + 1, 1)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 2, 6),
-				new Tuple2<Integer, Integer>(timeOffset + 3, 3)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 4, 10),
-				new Tuple2<Integer, Integer>(timeOffset + 5, 5)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 6, 14),
-				new Tuple2<Integer, Integer>(timeOffset + 7, 7)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 8, 18),
-				new Tuple2<Integer, Integer>(timeOffset + 9, 9)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 10, 22),
-				new Tuple2<Integer, Integer>(timeOffset + 11, 11)));
-
-		timeOffset = 25; // and back to the future ...
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 2, 2),
-				new Tuple2<Integer, Integer>(timeOffset + 1, 1)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 2, 6),
-				new Tuple2<Integer, Integer>(timeOffset + 3, 3)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 4, 10),
-				new Tuple2<Integer, Integer>(timeOffset + 5, 5)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 6, 14),
-				new Tuple2<Integer, Integer>(timeOffset + 7, 7)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 8, 18),
-				new Tuple2<Integer, Integer>(timeOffset + 9, 9)));
-		expected.add(StreamWindow.fromElements(
-				new Tuple2<Integer, Integer>(timeOffset + 10, 22),
-				new Tuple2<Integer, Integer>(timeOffset + 11, 11)));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	protected static void checkResults(List<StreamWindow<Integer>> expected,
-			List<StreamWindow<Integer>> actual) {
-
-		for (StreamWindow<Integer> sw : expected) {
-			Collections.sort(sw);
-		}
-
-		for (StreamWindow<Integer> sw : actual) {
-			Collections.sort(sw);
-		}
-
-		assertEquals(expected, actual);
-	}
-
-	@Test
-	public void testPreReduce2() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingTimeGroupedPreReducer<Integer> preReducer = new SlidingTimeGroupedPreReducer<Integer>(
-				reducer, serializer, key, 5, 2, new TimestampWrapper<Integer>(
-						new Timestamp<Integer>() {
-
-							private static final long serialVersionUID = 1L;
-
-							@Override
-							public long getTimestamp(Integer value) {
-								return value;
-							}
-						}, 1));
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.emitWindow(collector);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(9);
-		preReducer.store(10);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1, 2));
-		expected.add(StreamWindow.fromElements(4, 6));
-		expected.add(StreamWindow.fromElements(12, 8));
-		expected.add(StreamWindow.fromElements(18, 12));
-		expected.add(StreamWindow.fromElements(24, 16));
-		expected.add(StreamWindow.fromElements(30, 20));
-
-		checkResults(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce3() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingTimeGroupedPreReducer<Integer> preReducer = new SlidingTimeGroupedPreReducer<Integer>(
-				reducer, serializer, key, 6, 3, new TimestampWrapper<Integer>(
-						new Timestamp<Integer>() {
-
-							private static final long serialVersionUID = 1L;
-
-							@Override
-							public long getTimestamp(Integer value) {
-								return value;
-							}
-						}, 1));
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.store(3);
-		preReducer.emitWindow(collector);
-		preReducer.store(4);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.store(9);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(10);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(2, 4));
-		expected.add(StreamWindow.fromElements(9, 12));
-		expected.add(StreamWindow.fromElements(21, 18));
-		expected.add(StreamWindow.fromElements(30, 27));
-
-		checkResults(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce4() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingTimeGroupedPreReducer<Integer> preReducer = new SlidingTimeGroupedPreReducer<Integer>(
-				reducer, serializer, key, 3, 2, new TimestampWrapper<Integer>(
-						new Timestamp<Integer>() {
-
-							private static final long serialVersionUID = 1L;
-
-							@Override
-							public long getTimestamp(Integer value) {
-								return value;
-							}
-						}, 1));
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.emitWindow(collector);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(14);
-		preReducer.emitWindow(collector);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.emitWindow(collector);
-		preReducer.store(21);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-
-		preReducer.store(9);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(1, 2));
-		expected.add(StreamWindow.fromElements(3, 6));
-		expected.add(StreamWindow.fromElements(5, 10));
-		expected.add(StreamWindow.fromElements(7, 14));
-		expected.add(StreamWindow.fromElements(8));
-		expected.add(StreamWindow.fromElements(8));
-		expected.add(StreamWindow.fromElements(14));
-		expected.add(StreamWindow.fromElements(14));
-		expected.add(StreamWindow.fromElements(21));
-
-		checkResults(expected, collector.getCollected());
-	}
-
-	private static class SumReducer implements ReduceFunction<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1 + value2;
-		}
-
-	}
-
-	private static class TupleSumReducer implements ReduceFunction<Tuple2<Integer, Integer>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1, Tuple2<Integer, Integer> value2) throws Exception {
-			return new Tuple2<Integer, Integer>(value1.f0, value1.f1 + value2.f1);
-		}
-
-	}
-
-	public static class TupleModKey implements KeySelector<Tuple2<Integer, Integer>, Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		private int m;
-
-		public TupleModKey(int m) {
-			this.m = m;
-		}
-
-		@Override
-		public Integer getKey(Tuple2<Integer, Integer> value) throws Exception {
-			return value.f1 % m;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java
deleted file mode 100644
index 6a36c57..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/SlidingTimePreReducerTest.java
+++ /dev/null
@@ -1,324 +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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class SlidingTimePreReducerTest {
-
-	TypeSerializer<Integer> serializer = TypeExtractor.getForObject(1).createSerializer(null);
-	TypeInformation<Tuple2<Integer,Integer>> tupleType = TypeInfoParser.parse("Tuple2<Integer,Integer>");
-
-	ReduceFunction<Integer> reducer = new SumReducer();
-	ReduceFunction<Tuple2<Integer, Integer>> tupleReducer = new TupleSumReducer();
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testPreReduce1() throws Exception {
-		// This ensures that the buffer is properly cleared after a burst of elements by
-		// replaying the same sequence of elements with a later timestamp and expecting the same
-		// result.
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-
-		SlidingTimePreReducer<Tuple2<Integer, Integer>> preReducer = new SlidingTimePreReducer<Tuple2<Integer, Integer>>(tupleReducer,
-				tupleType.createSerializer(new ExecutionConfig()), 3, 2, new TimestampWrapper<Tuple2<Integer, Integer>>(new Timestamp<Tuple2<Integer, Integer>>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public long getTimestamp(Tuple2<Integer, Integer> value) {
-						return value.f0;
-					}
-				}, 1));
-
-		int timeOffset = 0;
-
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 1, 1));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 2, 2));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 3, 3));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 4, 4));
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 5, 5));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 6, 6));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 7, 7));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 8, 8));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 9, 9));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 10, 10));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 11, 11));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 12, 12));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 13, 13));
-
-		// ensure that everything is cleared out
-		preReducer.evict(100);
-
-
-		timeOffset = 25; // a little while later...
-
-		// Repeat the same sequence, this should produce the same result
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 1, 1));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 2, 2));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 3, 3));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 4, 4));
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 5, 5));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 6, 6));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 7, 7));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 8, 8));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 9, 9));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 10, 10));
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 11, 11));
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 12, 12));
-		preReducer.emitWindow(collector);
-		preReducer.store(new Tuple2<Integer, Integer>(timeOffset + 13, 13));
-
-		List<StreamWindow<Tuple2<Integer, Integer>>> expected = new ArrayList<StreamWindow<Tuple2<Integer, Integer>>>();
-		timeOffset = 0; // rewind ...
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 1, 3)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 2, 9)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 4, 15)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 6, 21)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 8, 27)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 10, 33)));
-
-		timeOffset = 25; // and back to the future ...
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 1, 3)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 2, 9)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 4, 15)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 6, 21)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 8, 27)));
-		expected.add(StreamWindow.fromElements(new Tuple2<Integer, Integer>(timeOffset + 10, 33)));
-
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce2() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingTimePreReducer<Integer> preReducer = new SlidingTimePreReducer<Integer>(reducer,
-				serializer, 5, 2, new TimestampWrapper<Integer>(new Timestamp<Integer>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public long getTimestamp(Integer value) {
-						return value;
-					}
-				}, 1));
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.emitWindow(collector);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(9);
-		preReducer.store(10);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(3));
-		expected.add(StreamWindow.fromElements(10));
-		expected.add(StreamWindow.fromElements(20));
-		expected.add(StreamWindow.fromElements(30));
-		expected.add(StreamWindow.fromElements(40));
-		expected.add(StreamWindow.fromElements(50));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce3() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingTimePreReducer<Integer> preReducer = new SlidingTimePreReducer<Integer>(reducer,
-				serializer, 6, 3, new TimestampWrapper<Integer>(new Timestamp<Integer>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public long getTimestamp(Integer value) {
-						return value;
-					}
-				}, 1));
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.store(3);
-		preReducer.emitWindow(collector);
-		preReducer.store(4);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.store(9);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(10);
-		preReducer.store(11);
-		preReducer.store(12);
-		preReducer.emitWindow(collector);
-		preReducer.evict(3);
-		preReducer.store(13);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(6));
-		expected.add(StreamWindow.fromElements(21));
-		expected.add(StreamWindow.fromElements(39));
-		expected.add(StreamWindow.fromElements(57));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	@Test
-	public void testPreReduce4() throws Exception {
-		TestOutput<StreamWindow<Integer>> collector = new TestOutput<StreamWindow<Integer>>();
-
-		SlidingTimePreReducer<Integer> preReducer = new SlidingTimePreReducer<Integer>(reducer,
-				serializer, 3, 2, new TimestampWrapper<Integer>(new Timestamp<Integer>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public long getTimestamp(Integer value) {
-						return value;
-					}
-				}, 1));
-
-		preReducer.store(1);
-		preReducer.store(2);
-		preReducer.emitWindow(collector);
-		preReducer.store(3);
-		preReducer.store(4);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(5);
-		preReducer.store(6);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(7);
-		preReducer.store(8);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.emitWindow(collector);
-		preReducer.emitWindow(collector);
-		preReducer.evict(2);
-		preReducer.store(14);
-		preReducer.emitWindow(collector);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-		preReducer.emitWindow(collector);
-		preReducer.store(21);
-		preReducer.emitWindow(collector);
-		preReducer.evict(1);
-		preReducer.emitWindow(collector);
-
-		preReducer.store(9);
-
-		List<StreamWindow<Integer>> expected = new ArrayList<StreamWindow<Integer>>();
-		expected.add(StreamWindow.fromElements(3));
-		expected.add(StreamWindow.fromElements(9));
-		expected.add(StreamWindow.fromElements(15));
-		expected.add(StreamWindow.fromElements(21));
-		expected.add(StreamWindow.fromElements(8));
-		expected.add(StreamWindow.fromElements(8));
-		expected.add(StreamWindow.fromElements(14));
-		expected.add(StreamWindow.fromElements(14));
-		expected.add(StreamWindow.fromElements(21));
-
-		assertEquals(expected, collector.getCollected());
-	}
-
-	private static class SumReducer implements ReduceFunction<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1 + value2;
-		}
-
-	}
-
-	private static class TupleSumReducer implements ReduceFunction<Tuple2<Integer, Integer>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1, Tuple2<Integer, Integer> value2) throws Exception {
-			return new Tuple2<Integer, Integer>(value1.f0, value1.f1 + value2.f1);
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java
deleted file mode 100644
index 3aee288..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingGroupedPreReducerTest.java
+++ /dev/null
@@ -1,151 +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.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-import org.junit.Test;
-
-public class TumblingGroupedPreReducerTest {
-
-	TypeInformation<Tuple2<Integer, Integer>> type = TypeExtractor
-			.getForObject(new Tuple2<Integer, Integer>(1, 1));
-	TypeSerializer<Tuple2<Integer, Integer>> serializer = type.createSerializer(null);
-
-	KeySelector<Tuple2<Integer, Integer>, ?> key = KeySelectorUtil.getSelectorForKeys(
-			new Keys.ExpressionKeys<Tuple2<Integer, Integer>>(new int[] { 0 }, type), type, null);
-
-	Reducer reducer = new Reducer();
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testEmitWindow() throws Exception {
-
-		List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
-		inputs.add(new Tuple2<Integer, Integer>(1, 1));
-		inputs.add(new Tuple2<Integer, Integer>(0, 0));
-		inputs.add(new Tuple2<Integer, Integer>(1, -1));
-		inputs.add(new Tuple2<Integer, Integer>(1, -2));
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-		List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();
-
-		WindowBuffer<Tuple2<Integer, Integer>> wb = new TumblingGroupedPreReducer<Tuple2<Integer, Integer>>(
-				reducer, key, serializer);
-
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.emitWindow(collector);
-		wb.evict(2);
-
-		assertEquals(1, collected.size());
-
-		assertSetEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(1, 1),
-				new Tuple2<Integer, Integer>(0, 0)), collected.get(0));
-
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.store(serializer.copy(inputs.get(2)));
-
-		wb.store(serializer.copy(inputs.get(3)));
-
-		wb.emitWindow(collector);
-		wb.evict(4);
-
-		assertEquals(2, collected.size());
-
-		assertSetEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(3, -2),
-				new Tuple2<Integer, Integer>(0, 0)), collected.get(1));
-
-		// Test whether function is mutating inputs or not
-		assertEquals(2, reducer.allInputs.size());
-		assertEquals(reducer.allInputs.get(0), inputs.get(2));
-		assertEquals(reducer.allInputs.get(1), inputs.get(3));
-
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testEmitWindow2() throws Exception {
-
-		List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
-		inputs.add(new Tuple2<Integer, Integer>(1, 1));
-		inputs.add(new Tuple2<Integer, Integer>(0, 0));
-		inputs.add(new Tuple2<Integer, Integer>(1, -1));
-		inputs.add(new Tuple2<Integer, Integer>(1, -2));
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-		List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();
-
-		WindowBuffer<Tuple2<Integer, Integer>> wb = new TumblingGroupedPreReducer<Tuple2<Integer, Integer>>(
-				reducer, key, serializer).sequentialID();
-
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.emitWindow(collector);
-		wb.evict(2);
-		
-		assertSetEquals(StreamWindow.fromElements(inputs.get(0), inputs.get(1)), collected.get(0));
-		
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.store(serializer.copy(inputs.get(2)));
-		wb.emitWindow(collector);
-		wb.evict(3);
-		
-		assertSetEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(2, 0), inputs.get(1)), collected.get(1));
-
-		
-	}
-
-	private static <T> void assertSetEquals(Collection<T> first, Collection<T> second) {
-		assertEquals(new HashSet<T>(first), new HashSet<T>(second));
-	}
-
-	@SuppressWarnings("serial")
-	private class Reducer implements ReduceFunction<Tuple2<Integer, Integer>> {
-
-		public List<Tuple2<Integer, Integer>> allInputs = new ArrayList<Tuple2<Integer, Integer>>();
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1,
-				Tuple2<Integer, Integer> value2) throws Exception {
-			allInputs.add(value2);
-			value1.f0 = value1.f0 + value2.f0;
-			value1.f1 = value1.f1 + value2.f1;
-			return value1;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java
deleted file mode 100644
index 3e537a5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/windowbuffer/TumblingPreReducerTest.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.windowbuffer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.windowing.StreamWindow;
-import org.apache.flink.streaming.api.windowing.windowbuffer.BasicWindowBufferTest.TestOutput;
-import org.junit.Test;
-
-public class TumblingPreReducerTest {
-
-	TypeSerializer<Tuple2<Integer, Integer>> serializer = TypeExtractor.getForObject(
-			new Tuple2<Integer, Integer>(1, 1)).createSerializer(null);
-
-	Reducer reducer = new Reducer();
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testEmitWindow() throws Exception {
-
-		List<Tuple2<Integer, Integer>> inputs = new ArrayList<Tuple2<Integer, Integer>>();
-		inputs.add(new Tuple2<Integer, Integer>(1, 1));
-		inputs.add(new Tuple2<Integer, Integer>(2, 0));
-		inputs.add(new Tuple2<Integer, Integer>(3, -1));
-		inputs.add(new Tuple2<Integer, Integer>(4, -2));
-
-		TestOutput<StreamWindow<Tuple2<Integer, Integer>>> collector = new TestOutput<StreamWindow<Tuple2<Integer, Integer>>>();
-		List<StreamWindow<Tuple2<Integer, Integer>>> collected = collector.getCollected();
-
-		WindowBuffer<Tuple2<Integer, Integer>> wb = new TumblingPreReducer<Tuple2<Integer, Integer>>(
-				reducer, serializer);
-
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-
-		wb.emitWindow(collector);
-		wb.evict(2);
-
-		assertEquals(1, collected.size());
-		assertEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(3, 1)),
-				collected.get(0));
-
-		wb.store(serializer.copy(inputs.get(0)));
-		wb.store(serializer.copy(inputs.get(1)));
-		wb.store(serializer.copy(inputs.get(2)));
-
-		wb.store(serializer.copy(inputs.get(3)));
-
-		wb.emitWindow(collector);
-		wb.evict(4);
-
-		assertEquals(2, collected.size());
-		assertEquals(StreamWindow.fromElements(new Tuple2<Integer, Integer>(10, -2)),
-				collected.get(1));
-
-		// Test whether function is mutating inputs or not
-		assertEquals(4, reducer.allInputs.size());
-		assertEquals(reducer.allInputs.get(0), inputs.get(1));
-		assertEquals(reducer.allInputs.get(1), inputs.get(1));
-		assertEquals(reducer.allInputs.get(2), inputs.get(2));
-		assertEquals(reducer.allInputs.get(3), inputs.get(3));
-
-	}
-
-	@SuppressWarnings("serial")
-	private class Reducer implements ReduceFunction<Tuple2<Integer, Integer>> {
-
-		public List<Tuple2<Integer, Integer>> allInputs = new ArrayList<Tuple2<Integer, Integer>>();
-
-		@Override
-		public Tuple2<Integer, Integer> reduce(Tuple2<Integer, Integer> value1,
-				Tuple2<Integer, Integer> value2) throws Exception {
-			allInputs.add(value2);
-			value1.f0 = value1.f0 + value2.f0;
-			value1.f1 = value1.f1 + value2.f1;
-			return value1;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
index 99d45bb..4c73e44 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -17,32 +17,35 @@
 
 package org.apache.flink.streaming.examples.ml;
 
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.WindowMapFunction;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
 import org.apache.flink.streaming.api.functions.co.CoMapFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.helper.Time;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.util.Collector;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * Skeleton for incremental machine learning algorithm consisting of a
  * pre-computed model, which gets updated for the new inputs and new input data
  * for which the job provides predictions.
- * <p/>
+ *
  * <p>
  * This may serve as a base of a number of algorithms, e.g. updating an
  * incremental Alternating Least Squares model while also providing the
  * predictions.
- * </p>
- * <p/>
+ *
  * <p/>
  * This example shows how to use:
  * <ul>
- * <li>Connected streams
- * <li>CoFunctions
- * <li>Tuple data types
+ *   <li>Connected streams
+ *   <li>CoFunctions
+ *   <li>Tuple data types
  * </ul>
  */
 public class IncrementalLearningSkeleton {
@@ -61,12 +64,16 @@ public class IncrementalLearningSkeleton {
 		}
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
 		trainingData = env.addSource(new FiniteTrainingDataSource());
 		newData = env.addSource(new FiniteNewDataSource());
 
 		// build new model on every second of new data
-		DataStream<Double[]> model = trainingData.window(Time.of(5000, new LinearTimestamp()))
-				.mapWindow(new PartialModelBuilder()).flatten();
+		DataStream<Double[]> model = trainingData
+				.extractTimestamp(new LinearTimestamp())
+				.timeWindowAll(Time.of(5000, TimeUnit.MILLISECONDS))
+				.apply(new PartialModelBuilder());
 
 		// use partial model for newData
 		DataStream<Integer> prediction = newData.connect(model).map(new Predictor());
@@ -140,21 +147,32 @@ public class IncrementalLearningSkeleton {
 		}
 	}
 
-	public static class LinearTimestamp implements Timestamp<Integer> {
+	public static class LinearTimestamp implements TimestampExtractor<Integer> {
 		private static final long serialVersionUID = 1L;
 
 		private long counter = 0L;
 
 		@Override
-		public long getTimestamp(Integer value) {
+		public long extractTimestamp(Integer element, long currentTimestamp) {
 			return counter += 10L;
 		}
+
+		@Override
+		public long emitWatermark(Integer element, long currentTimestamp) {
+			return counter - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+
 	}
 
 	/**
 	 * Builds up-to-date partial models on new training data.
 	 */
-	public static class PartialModelBuilder implements WindowMapFunction<Integer, Double[]> {
+	public static class PartialModelBuilder implements AllWindowFunction<Integer, Double[], TimeWindow> {
 		private static final long serialVersionUID = 1L;
 
 		protected Double[] buildPartialModel(Iterable<Integer> values) {
@@ -162,7 +180,7 @@ public class IncrementalLearningSkeleton {
 		}
 
 		@Override
-		public void mapWindow(Iterable<Integer> values, Collector<Double[]> out) throws Exception {
+		public void apply(TimeWindow window, Iterable<Integer> values, Collector<Double[]> out) throws Exception {
 			out.collect(buildPartialModel(values));
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
index dedc5ee..8a6cd88 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
@@ -19,15 +19,13 @@ package org.apache.flink.streaming.examples.ml.util;
 
 public class IncrementalLearningSkeletonData {
 
-	public static final String RESULTS = "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n"
-			+ "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
-			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
-			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
-			"0\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n"
-			+ "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" +
-			"1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" +
-			"1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" +
-			"1\n";
+	public static final String RESULTS = "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" +
+			"1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n";
 
 	private IncrementalLearningSkeletonData() {
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
index 4730cc1..950b0f5 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
@@ -18,11 +18,14 @@
 package org.apache.flink.streaming.examples.windowing;
 
 import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.policy.CentralActiveTrigger;
-import org.apache.flink.streaming.api.windowing.policy.TumblingEvictionPolicy;
+import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -37,37 +40,36 @@ public class SessionWindowing {
 		}
 
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		env.setParallelism(2);
 
-		final List<Tuple3<String, Long, Integer>> input = new ArrayList<Tuple3<String, Long, Integer>>();
+		final List<Tuple3<String, Long, Integer>> input = new ArrayList<>();
 
-		input.add(new Tuple3<String, Long, Integer>("a", 1L, 1));
-		input.add(new Tuple3<String, Long, Integer>("b", 1L, 1));
-		input.add(new Tuple3<String, Long, Integer>("b", 3L, 1));
-		input.add(new Tuple3<String, Long, Integer>("b", 5L, 1));
-		input.add(new Tuple3<String, Long, Integer>("c", 6L, 1));
+		input.add(new Tuple3<>("a", 1L, 1));
+		input.add(new Tuple3<>("b", 1L, 1));
+		input.add(new Tuple3<>("b", 3L, 1));
+		input.add(new Tuple3<>("b", 5L, 1));
+		input.add(new Tuple3<>("c", 6L, 1));
 		// We expect to detect the session "a" earlier than this point (the old
 		// functionality can only detect here when the next starts)
-		input.add(new Tuple3<String, Long, Integer>("a", 10L, 1));
+		input.add(new Tuple3<>("a", 10L, 1));
 		// We expect to detect session "b" and "c" at this point as well
-		input.add(new Tuple3<String, Long, Integer>("c", 11L, 1));
+		input.add(new Tuple3<>("c", 11L, 1));
 
 		DataStream<Tuple3<String, Long, Integer>> source = env
-				.addSource(new SourceFunction<Tuple3<String, Long, Integer>>() {
+				.addSource(new EventTimeSourceFunction<Tuple3<String,Long,Integer>>() {
 					private static final long serialVersionUID = 1L;
 
 					@Override
 					public void run(SourceContext<Tuple3<String, Long, Integer>> ctx) throws Exception {
 						for (Tuple3<String, Long, Integer> value : input) {
-							// We sleep three seconds between every output so we
-							// can see whether we properly detect sessions
-							// before the next start for a specific id
-							ctx.collect(value);
+							ctx.collectWithTimestamp(value, value.f1);
+							ctx.emitWatermark(new Watermark(value.f1 - 1));
 							if (!fileOutput) {
 								System.out.println("Collected: " + value);
-								Thread.sleep(3000);
 							}
 						}
+						ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
 					}
 
 					@Override
@@ -76,10 +78,11 @@ public class SessionWindowing {
 				});
 
 		// We create sessions for each id with max timeout of 3 time units
-		DataStream<Tuple3<String, Long, Integer>> aggregated = source.keyBy(0)
-				.window(new SessionTriggerPolicy(3L),
-						new TumblingEvictionPolicy<Tuple3<String, Long, Integer>>()).sum(2)
-				.flatten();
+		DataStream<Tuple3<String, Long, Integer>> aggregated = source
+				.keyBy(0)
+				.window(GlobalWindows.create())
+				.trigger(new SessionTrigger(3L))
+				.sum(2);
 
 		if (fileOutput) {
 			aggregated.writeAsText(outputPath);
@@ -90,55 +93,46 @@ public class SessionWindowing {
 		env.execute();
 	}
 
-	private static class SessionTriggerPolicy implements
-			CentralActiveTrigger<Tuple3<String, Long, Integer>> {
+	private static class SessionTrigger implements Trigger<Tuple3<String, Long, Integer>, GlobalWindow> {
 
 		private static final long serialVersionUID = 1L;
 
 		private volatile Long lastSeenEvent = 1L;
 		private Long sessionTimeout;
 
-		public SessionTriggerPolicy(Long sessionTimeout) {
+		public SessionTrigger(Long sessionTimeout) {
 			this.sessionTimeout = sessionTimeout;
 
 		}
 
 		@Override
-		public boolean notifyTrigger(Tuple3<String, Long, Integer> datapoint) {
-
-			Long eventTimestamp = datapoint.f1;
-			Long timeSinceLastEvent = eventTimestamp - lastSeenEvent;
+		public TriggerResult onElement(Tuple3<String, Long, Integer> element, long timestamp, GlobalWindow window, TriggerContext ctx) {
+			Long timeSinceLastEvent = timestamp - lastSeenEvent;
 
 			// Update the last seen event time
-			lastSeenEvent = eventTimestamp;
+			lastSeenEvent = timestamp;
+
+			ctx.registerWatermarkTimer(lastSeenEvent + sessionTimeout);
 
 			if (timeSinceLastEvent > sessionTimeout) {
-				return true;
+				return TriggerResult.FIRE_AND_PURGE;
 			} else {
-				return false;
+				return TriggerResult.CONTINUE;
 			}
 		}
 
 		@Override
-		public Object[] notifyOnLastGlobalElement(Tuple3<String, Long, Integer> datapoint) {
-			Long eventTimestamp = datapoint.f1;
-			Long timeSinceLastEvent = eventTimestamp - lastSeenEvent;
-
-			// Here we dont update the last seen event time because this data
-			// belongs to a different group
-
-			if (timeSinceLastEvent > sessionTimeout) {
-				return new Object[]{datapoint};
-			} else {
-				return null;
+		public TriggerResult onTime(long time, TriggerContext ctx) {
+			if (time - lastSeenEvent >= sessionTimeout) {
+				return TriggerResult.FIRE_AND_PURGE;
 			}
+			return TriggerResult.CONTINUE;
 		}
 
 		@Override
-		public SessionTriggerPolicy clone() {
-			return new SessionTriggerPolicy(sessionTimeout);
+		public SessionTrigger duplicate() {
+			return new SessionTrigger(sessionTimeout);
 		}
-
 	}
 
 	// *************************************************************************

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
index 55d48dd..a46ffd9 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
@@ -19,16 +19,20 @@ package org.apache.flink.streaming.examples.windowing;
 
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.windowing.helper.Delta;
-import org.apache.flink.streaming.api.windowing.helper.Time;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger;
 
 import java.util.Arrays;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 /**
  * An example of grouped stream windowing where different eviction and trigger
@@ -52,29 +56,37 @@ public class TopSpeedWindowing {
 		}
 
 		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 
 		@SuppressWarnings({"rawtypes", "serial"})
 		DataStream<Tuple4<Integer, Integer, Double, Long>> carData;
+
 		if (fileInput) {
 			carData = env.readTextFile(inputPath).map(new ParseCarData());
 		} else {
 			carData = env.addSource(CarSource.create(numOfCars));
 		}
-		DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds = carData.keyBy(0)
-				.window(Time.of(evictionSec * 1000, new CarTimestamp()))
-				.every(Delta.of(triggerMeters,
+
+		DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds = carData
+				.extractTimestamp(new CarTimestamp())
+				.keyBy(0)
+				.window(GlobalWindows.create())
+				.evictor(TimeEvictor.of(Time.of(evictionSec, TimeUnit.SECONDS)))
+				.trigger(DeltaTrigger.of(triggerMeters,
 						new DeltaFunction<Tuple4<Integer, Integer, Double, Long>>() {
 							private static final long serialVersionUID = 1L;
 
-
 							@Override
 							public double getDelta(
 									Tuple4<Integer, Integer, Double, Long> oldDataPoint,
 									Tuple4<Integer, Integer, Double, Long> newDataPoint) {
 								return newDataPoint.f2 - oldDataPoint.f2;
 							}
-						}, new Tuple4<Integer, Integer, Double, Long>(0, 0, 0d, 0l))).local().maxBy(1).flatten();
+						}))
+				.maxBy(1);
+
 		if (fileOutput) {
+			topSpeeds.print();
 			topSpeeds.writeAsText(outputPath);
 		} else {
 			topSpeeds.print();
@@ -143,17 +155,28 @@ public class TopSpeedWindowing {
 		public Tuple4<Integer, Integer, Double, Long> map(String record) {
 			String rawData = record.substring(1, record.length() - 1);
 			String[] data = rawData.split(",");
-			return new Tuple4<Integer, Integer, Double, Long>(Integer.valueOf(data[0]),
-					Integer.valueOf(data[1]), Double.valueOf(data[2]), Long.valueOf(data[3]));
+			return new Tuple4<>(Integer.valueOf(data[0]), Integer.valueOf(data[1]), Double.valueOf(data[2]), Long.valueOf(data[3]));
 		}
 	}
 
-	private static class CarTimestamp implements Timestamp<Tuple4<Integer, Integer, Double, Long>> {
+	private static class CarTimestamp implements TimestampExtractor<Tuple4<Integer, Integer, Double, Long>> {
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public long getTimestamp(Tuple4<Integer, Integer, Double, Long> value) {
-			return value.f3;
+		public long extractTimestamp(Tuple4<Integer, Integer, Double, Long> element,
+				long currentTimestamp) {
+			return element.f3;
+		}
+
+		@Override
+		public long emitWatermark(Tuple4<Integer, Integer, Double, Long> element,
+				long currentTimestamp) {
+			return element.f3 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0c1141ab/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
index 023a36a..04352d8 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
@@ -21,7 +21,9 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.examples.java.wordcount.util.WordCountData;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.helper.Count;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.examples.wordcount.WordCount;
 
 /**
@@ -70,11 +72,12 @@ public class WindowWordCount {
 		// split up the lines in pairs (2-tuples) containing: (word,1)
 		text.flatMap(new WordCount.Tokenizer())
 				// create windows of windowSize records slided every slideSize records
-				.window(Count.of(windowSize)).every(Count.of(slideSize))
+				.keyBy(0)
+				.window(GlobalWindows.create())
+				.evictor(CountEvictor.of(windowSize))
+				.trigger(CountTrigger.of(slideSize))
 				// group by the tuple field "0" and sum up tuple field "1"
-				.keyBy(0).sum(1)
-				// flatten the windows to a single stream
-				.flatten();
+				.sum(1);
 
 		// emit result
 		if (fileOutput) {