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

[1/9] flink git commit: [FLINK-2787] [core] Prevent instantiation of RemoteEnvironment when running a program through the command line.

Repository: flink
Updated Branches:
  refs/heads/master c04a77042 -> 846ad7064


[FLINK-2787] [core] Prevent instantiation of RemoteEnvironment when running a program through the command line.

This also cleans up the use of context environments.


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

Branch: refs/heads/master
Commit: 891db5e3e7037f5c889dae29d0182e6b50fc746c
Parents: c04a770
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 29 21:31:26 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 1 09:00:42 2015 +0200

----------------------------------------------------------------------
 .../org/apache/flink/client/LocalExecutor.java  |  9 +--
 .../org/apache/flink/client/program/Client.java |  6 +-
 .../client/program/ContextEnvironment.java      | 11 +++-
 .../program/OptimizerPlanEnvironment.java       | 12 ++--
 .../flink/client/program/PackagedProgram.java   | 15 ++---
 .../client/program/PreviewPlanEnvironment.java  |  7 ++-
 .../flink/api/java/ExecutionEnvironment.java    | 61 ++++++++++++--------
 .../apache/flink/api/java/LocalEnvironment.java | 27 ++++-----
 .../flink/api/java/RemoteEnvironment.java       | 38 +++++++++---
 9 files changed, 109 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
index 7928e53..25da5c7 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java
@@ -20,7 +20,6 @@ package org.apache.flink.client;
 
 import java.util.List;
 
-import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.Plan;
@@ -31,7 +30,6 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.optimizer.DataStatistics;
 import org.apache.flink.optimizer.Optimizer;
 import org.apache.flink.optimizer.dag.DataSinkNode;
@@ -63,7 +61,7 @@ public class LocalExecutor extends PlanExecutor {
 	private LocalFlinkMiniCluster flink;
 
 	/** Custom user configuration for the execution */
-	private Configuration configuration;
+	private final Configuration configuration;
 
 	/** Config value for how many slots to provide in the local cluster */
 	private int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
@@ -78,11 +76,6 @@ public class LocalExecutor extends PlanExecutor {
 	}
 
 	public LocalExecutor(Configuration conf) {
-		if (!ExecutionEnvironment.localExecutionIsAllowed()) {
-			throw new InvalidProgramException(
-					"The LocalEnvironment cannot be used when submitting a program through a client.");
-		}
-
 		this.configuration = conf != null ? conf : new Configuration();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
index 6c886fe..c72681d 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java
@@ -283,14 +283,13 @@ public class Client {
 		else if (prog.isUsingInteractiveMode()) {
 			LOG.info("Starting program in interactive mode");
 			ContextEnvironment.setAsContext(this, prog.getAllLibraries(), prog.getUserCodeClassLoader(), parallelism, true);
-			ContextEnvironment.enableLocalExecution(false);
 
 			// invoke here
 			try {
 				prog.invokeInteractiveModeForExecution();
 			}
 			finally {
-				ContextEnvironment.enableLocalExecution(true);
+				ContextEnvironment.unsetContext();
 			}
 
 			return JobExecutionResult.fromJobSubmissionResult(new JobSubmissionResult(lastJobID));
@@ -310,14 +309,13 @@ public class Client {
 		else if (prog.isUsingInteractiveMode()) {
 			LOG.info("Starting program in interactive mode");
 			ContextEnvironment.setAsContext(this, prog.getAllLibraries(), prog.getUserCodeClassLoader(), parallelism, false);
-			ContextEnvironment.enableLocalExecution(false);
 
 			// invoke here
 			try {
 				prog.invokeInteractiveModeForExecution();
 			}
 			finally {
-				ContextEnvironment.enableLocalExecution(true);
+				ContextEnvironment.unsetContext();
 			}
 
 			return new JobSubmissionResult(lastJobID);

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
index ad14a06..e33a05d 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
@@ -115,12 +115,17 @@ public class ContextEnvironment extends ExecutionEnvironment {
 		initializeContextEnvironment(factory);
 	}
 	
-	protected static void enableLocalExecution(boolean enabled) {
-		ExecutionEnvironment.enableLocalExecution(enabled);
+	static void unsetContext() {
+		resetContextEnvironment();
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
+
+	/**
+	 * The factory that instantiates the environment to be used when running jobs that are
+	 * submitted through a pre-configured client connection.
+	 * This happens for example when a job is submitted from the command line.
+	 */
 	public static class ContextEnvironmentFactory implements ExecutionEnvironmentFactory {
 		
 		private final Client client;

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
index c9c3b45..a5a9362 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
@@ -66,8 +66,7 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment {
 	}
 
 	public FlinkPlan getOptimizedPlan(PackagedProgram prog) throws ProgramInvocationException {
-		setAsContext();
-
+		
 		// temporarily write syserr and sysout to a byte array.
 		PrintStream originalOut = System.out;
 		PrintStream originalErr = System.err;
@@ -75,8 +74,9 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment {
 		System.setOut(new PrintStream(baos));
 		ByteArrayOutputStream baes = new ByteArrayOutputStream();
 		System.setErr(new PrintStream(baes));
+
+		setAsContext();
 		try {
-			ContextEnvironment.enableLocalExecution(false);
 			prog.invokeInteractiveModeForExecution();
 		}
 		catch (ProgramInvocationException e) {
@@ -91,7 +91,7 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment {
 			}
 		}
 		finally {
-			ContextEnvironment.enableLocalExecution(true);
+			unsetAsContext();
 			System.setOut(originalOut);
 			System.setErr(originalErr);
 			System.err.println(baes);
@@ -115,6 +115,10 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment {
 		};
 		initializeContextEnvironment(factory);
 	}
+	
+	private void unsetAsContext() {
+		resetContextEnvironment();
+	}
 
 	// ------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java
index 091a959..0cd4d07 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java
@@ -259,7 +259,6 @@ public class PackagedProgram {
 			PreviewPlanEnvironment env = new PreviewPlanEnvironment();
 			env.setAsContext();
 			try {
-				ContextEnvironment.enableLocalExecution(false);
 				invokeInteractiveModeForExecution();
 			}
 			catch (ProgramInvocationException e) {
@@ -276,7 +275,7 @@ public class PackagedProgram {
 				}
 			}
 			finally {
-				ContextEnvironment.enableLocalExecution(true);
+				env.unsetAsContext();
 			}
 			
 			if (env.previewPlan != null) {
@@ -292,12 +291,8 @@ public class PackagedProgram {
 
 		PlanJSONDumpGenerator jsonGen = new PlanJSONDumpGenerator();
 		StringWriter string = new StringWriter(1024);
-		PrintWriter pw = null;
-		try {
-			pw = new PrintWriter(string);
+		try (PrintWriter pw = new PrintWriter(string)) {
 			jsonGen.dumpPactPlanAsJSON(previewPlan, pw);
-		} finally {
-			pw.close();
 		}
 		return string.toString();
 
@@ -455,9 +450,9 @@ public class PackagedProgram {
 	}
 
 	private static String getEntryPointClassNameFromJar(File jarFile) throws ProgramInvocationException {
-		JarFile jar = null;
-		Manifest manifest = null;
-		String className = null;
+		JarFile jar;
+		Manifest manifest;
+		String className;
 
 		// Open jar file
 		try {

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-clients/src/main/java/org/apache/flink/client/program/PreviewPlanEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PreviewPlanEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/PreviewPlanEnvironment.java
index c5ced37..0051e60 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/PreviewPlanEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/PreviewPlanEnvironment.java
@@ -57,7 +57,7 @@ public final class PreviewPlanEnvironment extends ExecutionEnvironment {
 	}
 
 	@Override
-	public void startNewSession() throws Exception {
+	public void startNewSession() {
 	}
 
 	public void setAsContext() {
@@ -69,6 +69,11 @@ public final class PreviewPlanEnvironment extends ExecutionEnvironment {
 		};
 		initializeContextEnvironment(factory);
 	}
+	
+	public void unsetAsContext() {
+		resetContextEnvironment();
+	}
+	
 
 	public void setPreview(String preview) {
 		this.preview = preview;

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index 0f61d88..084e608 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -103,9 +103,6 @@ public abstract class ExecutionEnvironment {
 	/** The default parallelism used by local environments */
 	private static int defaultLocalDop = Runtime.getRuntime().availableProcessors();
 	
-	/** flag to disable local executor when using the ContextEnvironment */
-	private static boolean allowLocalExecution = true;
-	
 	// --------------------------------------------------------------------------------------------
 
 	private final List<DataSink<?>> sinks = new ArrayList<DataSink<?>>();
@@ -1127,9 +1124,7 @@ public abstract class ExecutionEnvironment {
 	 * @return A local execution environment with the specified parallelism.
 	 */
 	public static LocalEnvironment createLocalEnvironment(Configuration customConfiguration) {
-		LocalEnvironment lee = new LocalEnvironment();
-		lee.setConfiguration(customConfiguration);
-		return lee;
+		return new LocalEnvironment(customConfiguration);
 	}
 	
 	/**
@@ -1159,16 +1154,15 @@ public abstract class ExecutionEnvironment {
 	 *
 	 * @param host The host name or address of the master (JobManager), where the program should be executed.
 	 * @param port The port of the master (JobManager), where the program should be executed.
-	 * @param clientConfiguration Pass a custom configuration to the Client.
+	 * @param clientConfiguration Configuration used by the client that connects to the cluster.
 	 * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses
 	 *                 user-defined functions, user-defined input formats, or any libraries, those must be
 	 *                 provided in the JAR files.
 	 * @return A remote environment that executes the program on a cluster.
 	 */
-	public static ExecutionEnvironment createRemoteEnvironment(String host, int port, Configuration clientConfiguration, String... jarFiles) {
-		RemoteEnvironment rec = new RemoteEnvironment(host, port, jarFiles);
-		rec.setClientConfiguration(clientConfiguration);
-		return rec;
+	public static ExecutionEnvironment createRemoteEnvironment(
+			String host, int port, Configuration clientConfiguration, String... jarFiles) {
+		return new RemoteEnvironment(host, port, clientConfiguration, jarFiles);
 	}
 
 	/**
@@ -1201,23 +1195,40 @@ public abstract class ExecutionEnvironment {
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	//  Methods to control the context and local environments for execution from packaged programs
+	//  Methods to control the context environment and creation of explicit environments other
+	//  than the context environment
 	// --------------------------------------------------------------------------------------------
-	
+
+	/**
+	 * Sets a context environment factory, that creates the context environment for running programs
+	 * with pre-configured environments. Examples are running programs from the command line, and
+	 * running programs in the Scala shell.
+	 * 
+	 * <p>When the context environment factors is set, no other environments can be explicitly used.
+	 * 
+	 * @param ctx The context environment factory.
+	 */
 	protected static void initializeContextEnvironment(ExecutionEnvironmentFactory ctx) {
-		contextEnvironmentFactory = ctx;
+		contextEnvironmentFactory = Preconditions.checkNotNull(ctx);
 	}
-	
-	protected static boolean isContextEnvironmentSet() {
-		return contextEnvironmentFactory != null;
-	}
-	
-	protected static void enableLocalExecution(boolean enabled) {
-		allowLocalExecution = enabled;
-	}
-	
-	public static boolean localExecutionIsAllowed() {
-		return allowLocalExecution;
+
+	/**
+	 * Un-sets the context environment factory. After this method is called, the call to
+	 * {@link #getExecutionEnvironment()} will again return a default local execution environment, and
+	 * it is possible to explicitly instantiate the LocalEnvironment and the RemoteEnvironment.
+	 */
+	protected static void resetContextEnvironment() {
+		contextEnvironmentFactory = null;
 	}
 
+	/**
+	 * Checks whether it is currently permitted to explicitly instantiate a LocalEnvironment
+	 * or a RemoteEnvironment.
+	 * 
+	 * @return True, if it is possible to explicitly instantiate a LocalEnvironment or a
+	 *         RemoteEnvironment, false otherwise.
+	 */
+	public static boolean areExplicitEnvironmentsAllowed() {
+		return contextEnvironmentFactory == null;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
index 8163244..93050e5 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/LocalEnvironment.java
@@ -29,17 +29,17 @@ import org.apache.flink.configuration.Configuration;
  * An {@link ExecutionEnvironment} that runs the program locally, multi-threaded, in the JVM where the
  * environment is instantiated.
  * 
- * <p>When this environment is instantiated, it uses a default parallelism of {@code 1}. Teh default
- * parallelism can be set via {@link #setParallelism(int)}.</p>
+ * <p>When this environment is instantiated, it uses a default parallelism of {@code 1}. The default
+ * parallelism can be set via {@link #setParallelism(int)}.
  * 
  * <p>Local environments can also be instantiated through {@link ExecutionEnvironment#createLocalEnvironment()}
  * and {@link ExecutionEnvironment#createLocalEnvironment(int)}. The former version will pick a
- * default parallelism equal to the number of hardware contexts in the local machine.</p>
+ * default parallelism equal to the number of hardware contexts in the local machine.
  */
 public class LocalEnvironment extends ExecutionEnvironment {
 	
 	/** The user-defined configuration for the local execution */
-	private Configuration configuration;
+	private final Configuration configuration;
 
 	/** Create lazily upon first use */
 	private PlanExecutor executor;
@@ -53,20 +53,21 @@ public class LocalEnvironment extends ExecutionEnvironment {
 	 * Creates a new local environment.
 	 */
 	public LocalEnvironment() {
-		if (!ExecutionEnvironment.localExecutionIsAllowed()) {
-			throw new InvalidProgramException("The LocalEnvironment cannot be used when submitting a program through a client.");
-		}
-		this.configuration = new Configuration();
+		this(new Configuration());
 	}
 
 	/**
-	 * Sets a configuration used to configure the local Flink executor.
-	 * If {@code null} is passed, then the default configuration will be used.
+	 * Creates a new local environment that configures its local executor with the given configuration.
 	 * 
-	 * @param customConfiguration The configuration to be used for the local execution.
+	 * @param config The configuration used to configure the local executor.
 	 */
-	public void setConfiguration(Configuration customConfiguration) {
-		this.configuration = customConfiguration != null ? customConfiguration : new Configuration();
+	public LocalEnvironment(Configuration config) {
+		if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) {
+			throw new InvalidProgramException(
+					"The LocalEnvironment cannot be instantiated when running in a pre-defined context " +
+							"(such as Command Line Client, Scala Shell, or TestEnvironment)");
+		}
+		this.configuration = config == null ? new Configuration() : config;
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/891db5e3/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
index 63f59d3..5a4aa1e 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/RemoteEnvironment.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java;
 
+import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.Plan;
@@ -32,7 +33,7 @@ import org.apache.flink.configuration.Configuration;
  * <p>Many programs executed via the remote environment depend on additional classes. Such classes
  * may be the classes of functions (transformation, aggregation, ...) or libraries. Those classes
  * must be attached to the remote environment as JAR files, to allow the environment to ship the
- * classes into the cluster for the distributed execution.</p>
+ * classes into the cluster for the distributed execution.
  */
 public class RemoteEnvironment extends ExecutionEnvironment {
 	
@@ -45,11 +46,11 @@ public class RemoteEnvironment extends ExecutionEnvironment {
 	/** The jar files that need to be attached to each job */
 	private final String[] jarFiles;
 
-	/** The remote executor lazily created upon first use */
-	private PlanExecutor executor;
-	
+	/** The configuration used by the client that connects to the cluster */
 	private Configuration clientConfiguration;
 	
+	/** The remote executor lazily created upon first use */
+	private PlanExecutor executor;
 	
 	/** Optional shutdown hook, used in session mode to eagerly terminate the last session */
 	private Thread shutdownHook;
@@ -58,7 +59,7 @@ public class RemoteEnvironment extends ExecutionEnvironment {
 	 * Creates a new RemoteEnvironment that points to the master (JobManager) described by the
 	 * given host name and port.
 	 * 
-	 * <p>Each program execution will have all the given JAR files in its classpath.</p>
+	 * <p>Each program execution will have all the given JAR files in its classpath.
 	 * 
 	 * @param host The host name or address of the master (JobManager), where the program should be executed.
 	 * @param port The port of the master (JobManager), where the program should be executed. 
@@ -67,6 +68,28 @@ public class RemoteEnvironment extends ExecutionEnvironment {
 	 *                 provided in the JAR files.
 	 */	
 	public RemoteEnvironment(String host, int port, String... jarFiles) {
+		this(host, port, null, jarFiles);
+	}
+
+	/**
+	 * Creates a new RemoteEnvironment that points to the master (JobManager) described by the
+	 * given host name and port.
+	 * 
+	 * <p>Each program execution will have all the given JAR files in its classpath.
+	 *
+	 * @param host The host name or address of the master (JobManager), where the program should be executed.
+	 * @param port The port of the master (JobManager), where the program should be executed.
+	 * @param clientConfig The configuration used by the client that connects to the cluster.
+	 * @param jarFiles The JAR files with code that needs to be shipped to the cluster. If the program uses
+	 *                 user-defined functions, user-defined input formats, or any libraries, those must be
+	 *                 provided in the JAR files.
+	 */
+	public RemoteEnvironment(String host, int port, Configuration clientConfig, String... jarFiles) {
+		if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) {
+			throw new InvalidProgramException(
+					"The RemoteEnvironment cannot be instantiated when running in a pre-defined context " +
+							"(such as Command Line Client, Scala Shell, or TestEnvironment)");
+		}
 		if (host == null) {
 			throw new NullPointerException("Host must not be null.");
 		}
@@ -77,6 +100,7 @@ public class RemoteEnvironment extends ExecutionEnvironment {
 		this.host = host;
 		this.port = port;
 		this.jarFiles = jarFiles;
+		this.clientConfiguration = clientConfig == null ? new Configuration() : clientConfig;
 	}
 
 	// ------------------------------------------------------------------------
@@ -170,10 +194,6 @@ public class RemoteEnvironment extends ExecutionEnvironment {
 				(getParallelism() == -1 ? "default" : getParallelism()) + ") : " + getIdString();
 	}
 	
-	public void setClientConfiguration(Configuration clientConfiguration) {
-		this.clientConfiguration = clientConfiguration;
-	}
-	
 	// ------------------------------------------------------------------------
 	//  Shutdown hooks and reapers
 	// ------------------------------------------------------------------------


[6/9] flink git commit: [hotfix] Simplify new windowing API

Posted by se...@apache.org.
[hotfix] Simplify new windowing API

Before, there would be three different window() methods on
KeyedDataStream: one that takes two policies, one that takes one policy
and one that takes a window assigner.

Now, there is only one window() method that takes a window assigner and
creates a KeyedWindowDataStream.

For conveniece, there are two methods timeWindows() that take either one
argument (tumbling windows) or two arguments (sliding windows). These
create a KeyedWindowDataStream with either a SlidingWindows or
TumblingWindows assigner.

When the window operator is created we pick the optimized aligned time
windows operator if the combination of window assigner/trigger/evictor
allows it.

All of this behaviour is verified in tests.

This closes #1195


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

Branch: refs/heads/master
Commit: 5623c15b0933487aea11fa8962feef29433133b7
Parents: 937793e
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Sep 29 20:22:11 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 1 11:04:00 2015 +0200

----------------------------------------------------------------------
 .../api/datastream/KeyedDataStream.java         |  64 +++--
 .../KeyedTriggerWindowDataStream.java           | 255 -------------------
 .../api/datastream/KeyedWindowDataStream.java   | 237 +++++++++++++----
 .../windowing/ReduceKeyedWindowFunction.java    |  70 +++++
 .../api/windowing/evictors/TimeEvictor.java     |   6 +
 .../api/windowing/time/AbstractTime.java        |  95 +++++++
 .../streaming/api/windowing/time/EventTime.java |  62 +++++
 .../api/windowing/time/ProcessingTime.java      |  63 +++++
 .../streaming/api/windowing/time/Time.java      |  66 +++++
 .../ContinuousProcessingTimeTrigger.java        |   6 +
 .../triggers/ContinuousWatermarkTrigger.java    |   6 +
 .../windowpolicy/AbstractTimePolicy.java        | 109 --------
 .../api/windowing/windowpolicy/Count.java       |  58 -----
 .../api/windowing/windowpolicy/Delta.java       |  68 -----
 .../api/windowing/windowpolicy/EventTime.java   |  64 -----
 .../windowing/windowpolicy/ProcessingTime.java  |  65 -----
 .../api/windowing/windowpolicy/Time.java        |  68 -----
 .../windowing/windowpolicy/WindowPolicy.java    |  57 -----
 .../operators/windowing/PolicyToOperator.java   | 239 -----------------
 .../windowing/EvictingWindowOperatorTest.java   |   4 +-
 .../windowing/PolicyWindowTranslationTest.java  | 216 ----------------
 .../windowing/TimeWindowTranslationTest.java    |  97 +++++++
 .../windowing/TriggerWindowTranslationTest.java | 201 ---------------
 .../operators/windowing/WindowOperatorTest.java |  10 +-
 .../windowing/WindowTranslationTest.java        | 201 +++++++++++++++
 .../GroupedProcessingTimeWindowExample.java     |   4 +-
 26 files changed, 916 insertions(+), 1475 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
index ce105e5..2ae07b2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedDataStream.java
@@ -23,8 +23,14 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+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.windowpolicy.WindowPolicy;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
+import org.apache.flink.streaming.api.windowing.time.EventTime;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
@@ -92,34 +98,50 @@ public class KeyedDataStream<T, KEY> extends DataStream<T> {
 	// ------------------------------------------------------------------------
 
 	/**
-	 * Windows this data stream to a KeyedWindowDataStream, which evaluates windows over a key
-	 * grouped stream. The window is defined by a single policy.
+	 * Windows this {@code KeyedDataStream} into tumbling time windows.
+	 *
 	 * <p>
-	 * For time windows, these single-policy windows result in tumbling time windows.
-	 *     
-	 * @param policy The policy that defines the window.
-	 * @return The windows data stream. 
+	 * This is a shortcut for either {@code .window(TumblingTimeWindows.of(size))} or
+	 * {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic
+	 * set using
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
+	 *
+	 * @param size The size of the window.
 	 */
-	public KeyedWindowDataStream<T, KEY> window(WindowPolicy policy) {
-		return new KeyedWindowDataStream<T, KEY>(this, policy);
+	public KeyedWindowDataStream<T, KEY, TimeWindow> timeWindow(AbstractTime size) {
+		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+
+		if (actualSize instanceof EventTime) {
+			return window(TumblingTimeWindows.of(actualSize.toMilliseconds()));
+		} else {
+			return window(TumblingProcessingTimeWindows.of(actualSize.toMilliseconds()));
+		}
 	}
 
 	/**
-	 * Windows this data stream to a KeyedWindowDataStream, which evaluates windows over a key
-	 * grouped stream. The window is defined by a window policy, plus a slide policy.
+	 * Windows this {@code KeyedDataStream} into sliding time windows.
+	 *
 	 * <p>
-	 * For time windows, these slide policy windows result in sliding time windows.
-	 * 
-	 * @param window The policy that defines the window.
-	 * @param slide The additional policy defining the slide of the window. 
-	 * @return The windows data stream.
+	 * This is a shortcut for either {@code .window(SlidingTimeWindows.of(size, slide))} or
+	 * {@code .window(SlidingProcessingTimeWindows.of(size, slide))} depending on the time characteristic
+	 * set using
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
+	 *
+	 * @param size The size of the window.
 	 */
-	public KeyedWindowDataStream<T, KEY> window(WindowPolicy window, WindowPolicy slide) {
-		return new KeyedWindowDataStream<T, KEY>(this, window, slide);
+	public KeyedWindowDataStream<T, KEY, TimeWindow> timeWindow(AbstractTime size, AbstractTime slide) {
+		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+
+		if (actualSize instanceof EventTime) {
+			return window(SlidingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
+		} else {
+			return window(SlidingProcessingTimeWindows.of(actualSize.toMilliseconds(), actualSlide.toMilliseconds()));
+		}
 	}
 
 	/**
-	 * Windows this data stream to a {@code KeyedTriggerWindowDataStream}, which evaluates windows
+	 * Windows this data stream to a {@code KeyedWindowDataStream}, which evaluates windows
 	 * over a key grouped stream. Elements are put into windows by a {@link WindowAssigner}. The
 	 * grouping of elements is done both by key and by window.
 	 *
@@ -131,7 +153,7 @@ public class KeyedDataStream<T, KEY> extends DataStream<T> {
 	 * @param assigner The {@code WindowAssigner} that assigns elements to windows.
 	 * @return The trigger windows data stream.
 	 */
-	public <W extends Window> KeyedTriggerWindowDataStream<T, KEY, W> window(WindowAssigner<? super T, W> assigner) {
-		return new KeyedTriggerWindowDataStream<T, KEY, W>(this, assigner);
+	public <W extends Window> KeyedWindowDataStream<T, KEY, W> window(WindowAssigner<? super T, W> assigner) {
+		return new KeyedWindowDataStream<>(this, assigner);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedTriggerWindowDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedTriggerWindowDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedTriggerWindowDataStream.java
deleted file mode 100644
index 5b39775..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedTriggerWindowDataStream.java
+++ /dev/null
@@ -1,255 +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.commons.lang.SerializationUtils;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.Utils;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.api.windowing.evictors.Evictor;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.windowpolicy.ProcessingTime;
-import org.apache.flink.streaming.api.windowing.windowpolicy.WindowPolicy;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.PolicyToOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * A {@code KeyedTriggerWindowDataStream} 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
- * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. Window emission
- * is triggered based on a {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
- *
- * <p>
- * The windows are conceptually evaluated for each key individually, meaning windows can trigger at
- * different points for each key.
- *
- * <p>
- * If an {@link Evictor} is specified it will be used to evict elements from the window after
- * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window.
- * When using an evictor window performance will degrade significantly, since
- * pre-aggregation of window results cannot be used.
- *
- * <p>
- * Note that the {@code KeyedTriggerWindowDataStream} is purely and API construct, during runtime
- * the {@code KeyedTriggerWindowDataStream} will be collapsed together with the
- * {@code KeyedDataStream} and the operation over the window into one single operation.
- * 
- * @param <T> The type of elements in the stream.
- * @param <K> The type of the key by which elements are grouped.
- * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns the elements to.
- */
-public class KeyedTriggerWindowDataStream<T, K, W extends Window> {
-
-	/** The keyed data stream that is windowed by this stream */
-	private final KeyedDataStream<T, K> input;
-
-	/** The window assigner */
-	private final WindowAssigner<? super T, W> windowAssigner;
-
-	/** The trigger that is used for window evaluation/emission. */
-	private Trigger<? super T, ? super W> trigger;
-
-	/** The evictor that is used for evicting elements before window evaluation. */
-	private Evictor<? super T, ? super W> evictor;
-
-
-	public KeyedTriggerWindowDataStream(KeyedDataStream<T, K> input, WindowAssigner<? super T, W> windowAssigner) {
-		this.input = input;
-		this.windowAssigner = windowAssigner;
-		this.trigger = windowAssigner.getDefaultTrigger();
-	}
-
-	/**
-	 * Sets the {@code Trigger} that should be used to trigger window emission.
-	 */
-	public KeyedTriggerWindowDataStream<T, K, W> trigger(Trigger<? super T, ? super W> trigger) {
-		this.trigger = trigger;
-		return this;
-	}
-
-	/**
-	 * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
-	 *
-	 * <p>
-	 * Note: When using an evictor window performance will degrade significantly, since
-	 * pre-aggregation of window results cannot be used.
-	 */
-	public KeyedTriggerWindowDataStream<T, K, W> evictor(Evictor<? super T, ? super W> evictor) {
-		this.evictor = evictor;
-		return this;
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  Operations on the keyed windows
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Applies a reduce function to the window. The window function is called for each evaluation
-	 * of the window for each key individually. The output of the reduce function is interpreted
-	 * as a regular non-windowed stream.
-	 * <p>
-	 * This window will try and pre-aggregate data as much as the window policies permit. For example,
-	 * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
-	 * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide interval,
-	 * so a few elements are stored per key (one per slide interval).
-	 * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
-	 * aggregation tree.
-	 * 
-	 * @param function The reduce function.
-	 * @return The data stream that is the result of applying the reduce function to the window. 
-	 */
-	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "Reduce at " + callLocation;
-
-		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
-		if (result != null) {
-			return result;
-		}
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-		KeySelector<T, K> keySel = input.getKeySelector();
-
-		OneInputStreamOperator<T, T> operator;
-
-		if (evictor != null) {
-			operator = new EvictingWindowOperator<>(windowAssigner,
-					keySel,
-					new HeapWindowBuffer.Factory<T>(),
-					new ReduceWindowFunction<K, W, T>(function),
-					trigger,
-					evictor);
-
-		} else {
-			// we need to copy because we need our own instance of the pre aggregator
-			@SuppressWarnings("unchecked")
-			ReduceFunction<T> functionCopy = (ReduceFunction<T>) SerializationUtils.clone(function);
-
-			operator = new WindowOperator<>(windowAssigner,
-					keySel,
-					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
-					new ReduceWindowFunction<K, W, T>(function),
-					trigger);
-		}
-
-		return input.transform(opName, input.getType(), operator);
-	}
-
-	/**
-	 * Applies a window function to the 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.
-	 * <p>
-	 * 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.
-	 */
-	public <R> DataStream<R> mapWindow(KeyedWindowFunction<T, R, K, W> function) {
-		TypeInformation<T> inType = input.getType();
-		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
-				function, KeyedWindowFunction.class, true, true, inType, null, false);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "MapWindow at " + callLocation;
-
-		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
-		if (result != null) {
-			return result;
-		}
-
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-		KeySelector<T, K> keySel = input.getKeySelector();
-
-		OneInputStreamOperator<T, R> operator;
-
-		if (evictor != null) {
-			operator = new EvictingWindowOperator<>(windowAssigner,
-					keySel,
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger,
-					evictor);
-
-		} else {
-			operator = new WindowOperator<>(windowAssigner,
-					keySel,
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger);
-		}
-
-
-
-		return input.transform(opName, resultType, operator);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private <R> DataStream<R> createFastTimeOperatorIfValid(
-			Function function,
-			TypeInformation<R> resultType,
-			String functionName) {
-
-		WindowPolicy windowPolicy = null;
-		WindowPolicy slidePolicy = null;
-
-		if (windowAssigner instanceof SlidingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			SlidingProcessingTimeWindows timeWindows = (SlidingProcessingTimeWindows) windowAssigner;
-			windowPolicy = ProcessingTime.of(timeWindows.getSize(), TimeUnit.MILLISECONDS);
-			slidePolicy = ProcessingTime.of(timeWindows.getSlide(), TimeUnit.MILLISECONDS);
-		} else if (windowAssigner instanceof TumblingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			TumblingProcessingTimeWindows timeWindows = (TumblingProcessingTimeWindows) windowAssigner;
-			windowPolicy = ProcessingTime.of(timeWindows.getSize(), TimeUnit.MILLISECONDS);
-		}
-
-		if (windowPolicy == null) {
-			return null;
-		}
-
-		String opName = windowPolicy.toString(slidePolicy) + " of " + functionName;
-		KeySelector<T, K> keySel = input.getKeySelector();
-
-		OneInputStreamOperator<T, R> operator =
-				PolicyToOperator.createOperatorForPolicies(windowPolicy, slidePolicy, function, keySel);
-
-		return input.transform(opName, resultType, operator);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java
index e658bdd..ad7ca37 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedWindowDataStream.java
@@ -18,61 +18,98 @@
 
 package org.apache.flink.streaming.api.datastream;
 
+import org.apache.commons.lang.SerializationUtils;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.Utils;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.windowpolicy.WindowPolicy;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.evictors.Evictor;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.PolicyToOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
 
 /**
- * A KeyedWindowDataStream represents a data stream where elements are grouped by key, and 
- * for each key, the stream of elements is split into windows. The windows are conceptually
- * evaluated for each key individually, meaning windows can trigger at different points
- * for each key.
+ * A {@code KeyedWindowDataStream} 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
+ * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. Window emission
+ * is triggered based on a {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
+ *
+ * <p>
+ * The windows are conceptually evaluated for each key individually, meaning windows can trigger at
+ * different points for each key.
+ *
  * <p>
- * In many cases, however, the windows are "aligned", meaning they trigger at the
- * same time for all keys. The most common example for that are the regular time windows.
+ * If an {@link Evictor} is specified it will be used to evict elements from the window after
+ * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window.
+ * When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ *
  * <p>
- * Note that the KeyedWindowDataStream is purely and API construct, during runtime the
- * KeyedWindowDataStream will be collapsed together with the KeyedDataStream and the operation
- * over the window into one single operation.
+ * Note that the {@code KeyedWindowDataStream} is purely and API construct, during runtime
+ * the {@code KeyedWindowDataStream} will be collapsed together with the
+ * {@code KeyedDataStream} and the operation over the window into one single operation.
  * 
  * @param <T> The type of elements in the stream.
  * @param <K> The type of the key by which elements are grouped.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns the elements to.
  */
-public class KeyedWindowDataStream<T, K> {
+public class KeyedWindowDataStream<T, K, W extends Window> {
 
 	/** The keyed data stream that is windowed by this stream */
 	private final KeyedDataStream<T, K> input;
 
-	/** The core window policy */
-	private final WindowPolicy windowPolicy;
+	/** The window assigner */
+	private final WindowAssigner<? super T, W> windowAssigner;
 
-	/** The optional additional slide policy */
-	private final WindowPolicy slidePolicy;
-	
-	
-	public KeyedWindowDataStream(KeyedDataStream<T, K> input, WindowPolicy windowPolicy) {
-		this(input, windowPolicy, null);
-	}
+	/** The trigger that is used for window evaluation/emission. */
+	private Trigger<? super T, ? super W> trigger;
+
+	/** The evictor that is used for evicting elements before window evaluation. */
+	private Evictor<? super T, ? super W> evictor;
 
-	public KeyedWindowDataStream(KeyedDataStream<T, K> input,
-								WindowPolicy windowPolicy, WindowPolicy slidePolicy) 
-	{
-		TimeCharacteristic time = input.getExecutionEnvironment().getStreamTimeCharacteristic();
 
+	public KeyedWindowDataStream(KeyedDataStream<T, K> input,
+			WindowAssigner<? super T, W> windowAssigner) {
 		this.input = input;
-		this.windowPolicy = windowPolicy.makeSpecificBasedOnTimeCharacteristic(time);
-		this.slidePolicy = slidePolicy == null ? null : slidePolicy.makeSpecificBasedOnTimeCharacteristic(time);
+		this.windowAssigner = windowAssigner;
+		this.trigger = windowAssigner.getDefaultTrigger();
 	}
-	
+
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	public KeyedWindowDataStream<T, K, W> trigger(Trigger<? super T, ? super W> trigger) {
+		this.trigger = trigger;
+		return this;
+	}
+
+	/**
+	 * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
+	 *
+	 * <p>
+	 * Note: When using an evictor window performance will degrade significantly, since
+	 * pre-aggregation of window results cannot be used.
+	 */
+	public KeyedWindowDataStream<T, K, W> evictor(Evictor<? super T, ? super W> evictor) {
+		this.evictor = evictor;
+		return this;
+	}
+
+
 	// ------------------------------------------------------------------------
 	//  Operations on the keyed windows
 	// ------------------------------------------------------------------------
@@ -94,7 +131,39 @@ public class KeyedWindowDataStream<T, K> {
 	 */
 	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
 		String callLocation = Utils.getCallLocationName();
-		return createWindowOperator(function, input.getType(), "Reduce at " + callLocation);
+		String udfName = "Reduce at " + callLocation;
+
+		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
+		if (result != null) {
+			return result;
+		}
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+		KeySelector<T, K> keySel = input.getKeySelector();
+
+		OneInputStreamOperator<T, T> operator;
+
+		if (evictor != null) {
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					keySel,
+					new HeapWindowBuffer.Factory<T>(),
+					new ReduceKeyedWindowFunction<K, W, T>(function),
+					trigger,
+					evictor);
+
+		} else {
+			// we need to copy because we need our own instance of the pre aggregator
+			@SuppressWarnings("unchecked")
+			ReduceFunction<T> functionCopy = (ReduceFunction<T>) SerializationUtils.clone(function);
+
+			operator = new WindowOperator<>(windowAssigner,
+					keySel,
+					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
+					new ReduceKeyedWindowFunction<K, W, T>(function),
+					trigger);
+		}
+
+		return input.transform(opName, input.getType(), operator);
 	}
 
 	/**
@@ -108,29 +177,107 @@ public class KeyedWindowDataStream<T, K> {
 	 * @param function The window function.
 	 * @return The data stream that is the result of applying the window function to the window.
 	 */
-	public <Result> DataStream<Result> mapWindow(KeyedWindowFunction<T, Result, K, Window> function) {
-		String callLocation = Utils.getCallLocationName();
-
+	public <R> DataStream<R> mapWindow(KeyedWindowFunction<T, R, K, W> function) {
 		TypeInformation<T> inType = input.getType();
-		TypeInformation<Result> resultType = TypeExtractor.getUnaryOperatorReturnType(
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
 				function, KeyedWindowFunction.class, true, true, inType, null, false);
 
-		return createWindowOperator(function, resultType, "KeyedWindowFunction at " + callLocation);
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "MapWindow at " + callLocation;
+
+		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		if (result != null) {
+			return result;
+		}
+
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+		KeySelector<T, K> keySel = input.getKeySelector();
+
+		OneInputStreamOperator<T, R> operator;
+
+		if (evictor != null) {
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					keySel,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor);
+
+		} else {
+			operator = new WindowOperator<>(windowAssigner,
+					keySel,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger);
+		}
+
+
+
+		return input.transform(opName, resultType, operator);
 	}
 
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------
-	
-	private <Result> DataStream<Result> createWindowOperator(
-			Function function, TypeInformation<Result> resultType, String functionName) {
 
-		String opName = windowPolicy.toString(slidePolicy) + " of " + functionName;
-		KeySelector<T, K> keySel = input.getKeySelector();
-		
-		OneInputStreamOperator<T, Result> operator =
-				PolicyToOperator.createOperatorForPolicies(windowPolicy, slidePolicy, function, keySel);
-		
-		return input.transform(opName, resultType, operator);
+	private <R> DataStream<R> createFastTimeOperatorIfValid(
+			Function function,
+			TypeInformation<R> resultType,
+			String functionName) {
+
+		if (windowAssigner instanceof SlidingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			SlidingProcessingTimeWindows timeWindows = (SlidingProcessingTimeWindows) windowAssigner;
+			final long windowLength = timeWindows.getSize();
+			final long windowSlide = timeWindows.getSlide();
+
+			String opName = "Fast " + timeWindows + " of " + functionName;
+
+			if (function instanceof ReduceFunction) {
+				@SuppressWarnings("unchecked")
+				ReduceFunction<T> reducer = (ReduceFunction<T>) function;
+
+				@SuppressWarnings("unchecked")
+				OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
+						new AggregatingProcessingTimeWindowOperator<>(
+								reducer, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+			else if (function instanceof KeyedWindowFunction) {
+				@SuppressWarnings("unchecked")
+				KeyedWindowFunction<T, R, K, Window> wf = (KeyedWindowFunction<T, R, K, Window>) function;
+
+				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
+						wf, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+		} else if (windowAssigner instanceof TumblingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			TumblingProcessingTimeWindows timeWindows = (TumblingProcessingTimeWindows) windowAssigner;
+			final long windowLength = timeWindows.getSize();
+			final long windowSlide = timeWindows.getSize();
+
+			String opName = "Fast " + timeWindows + " of " + functionName;
+
+			if (function instanceof ReduceFunction) {
+				@SuppressWarnings("unchecked")
+				ReduceFunction<T> reducer = (ReduceFunction<T>) function;
+
+				@SuppressWarnings("unchecked")
+				OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
+						new AggregatingProcessingTimeWindowOperator<>(
+								reducer, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+			else if (function instanceof KeyedWindowFunction) {
+				@SuppressWarnings("unchecked")
+				KeyedWindowFunction<T, R, K, Window> wf = (KeyedWindowFunction<T, R, K, Window>) function;
+
+				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
+						wf, input.getKeySelector(), windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+		}
+
+		return null;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.java
new file mode 100644
index 0000000..70627f0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceKeyedWindowFunction.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.functions.windowing;
+
+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.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+public class ReduceKeyedWindowFunction<K, W extends Window, T> extends RichKeyedWindowFunction<T, T, K, W> {
+	private static final long serialVersionUID = 1L;
+
+	private final ReduceFunction<T> reduceFunction;
+
+	public ReduceKeyedWindowFunction(ReduceFunction<T> reduceFunction) {
+		this.reduceFunction = reduceFunction;
+	}
+
+	@Override
+	public void setRuntimeContext(RuntimeContext ctx) {
+		super.setRuntimeContext(ctx);
+		FunctionUtils.setFunctionRuntimeContext(reduceFunction, ctx);
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		FunctionUtils.openFunction(reduceFunction, parameters);
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		FunctionUtils.closeFunction(reduceFunction);
+	}
+
+	@Override
+	public void evaluate(K k, W window, Iterable<T> values, Collector<T> out) throws Exception {
+		T result = null;
+
+		for (T v: values) {
+			if (result == null) {
+				result = v;
+			} else {
+				result = reduceFunction.reduce(result, v);
+			}
+		}
+
+		if (result != null) {
+			out.collect(result);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/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 450b132..c38100c 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
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.evictors;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
@@ -48,6 +49,11 @@ public class TimeEvictor<W extends Window> implements Evictor<Object, W> {
 		return "TimeEvictor(" + windowSize + ")";
 	}
 
+	@VisibleForTesting
+	public long getWindowSize() {
+		return windowSize;
+	}
+
 	public static <W extends Window> TimeEvictor<W> of(long windowSize) {
 		return new TimeEvictor<>(windowSize);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
new file mode 100644
index 0000000..1264c2a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.windowing.time;
+
+import org.apache.flink.streaming.api.TimeCharacteristic;
+
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public abstract class AbstractTime {
+
+	/** The time unit for this policy's time interval */
+	private final TimeUnit unit;
+	
+	/** The size of the windows generated by this policy */
+	private final long size;
+
+
+	protected AbstractTime(long size, TimeUnit unit) {
+		this.unit = checkNotNull(unit, "time unit may not be null");
+		this.size = size;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the time unit for this policy's time interval.
+	 * @return The time unit for this policy's time interval.
+	 */
+	public TimeUnit getUnit() {
+		return unit;
+	}
+
+	/**
+	 * Gets the length of this policy's time interval.
+	 * @return The length of this policy's time interval.
+	 */
+	public long getSize() {
+		return size;
+	}
+
+	/**
+	 * Converts the time interval to milliseconds.
+	 * @return The time interval in milliseconds.
+	 */
+	public long toMilliseconds() {
+		return unit.toMillis(size);
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	public abstract AbstractTime makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic characteristic);
+
+	@Override
+	public int hashCode() {
+		return 31 * (int) (size ^ (size >>> 32)) + unit.hashCode();
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj != null && obj.getClass() == getClass()) {
+			AbstractTime that = (AbstractTime) obj;
+			return this.size == that.size && this.unit.equals(that.unit);
+		}
+		else {
+			return false;
+		}
+	}
+
+	@Override
+	public String toString() {
+		return getClass().getSimpleName() + " (" + size + ' ' + unit.name() + ')';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/EventTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/EventTime.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/EventTime.java
new file mode 100644
index 0000000..6a4349c
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/EventTime.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.windowing.time;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The definition of an event time interval for windowing. See
+ * {@link org.apache.flink.streaming.api.TimeCharacteristic#EventTime} for a definition
+ * of event time.
+ */
+public final class EventTime extends AbstractTime {
+
+	/** Instantiation only via factory method */
+	private EventTime(long size, TimeUnit unit) {
+		super(size, unit);
+	}
+
+	@Override
+	public EventTime makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic characteristic) {
+		if (characteristic == TimeCharacteristic.EventTime || characteristic == TimeCharacteristic.IngestionTime) {
+			return this;
+		}
+		else {
+			throw new InvalidProgramException(
+					"Cannot use EventTime policy in a dataflow that runs on " + characteristic);
+		}
+	}
+	// ------------------------------------------------------------------------
+	//  Factory
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates an event time policy describing an event time interval.
+	 *
+	 * @param size The size of the generated windows.
+	 * @param unit The init (seconds, milliseconds) of the time interval.
+	 * @return The event time policy.
+	 */
+	public static EventTime of(long size, TimeUnit unit) {
+		return new EventTime(size, unit);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/ProcessingTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/ProcessingTime.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/ProcessingTime.java
new file mode 100644
index 0000000..4be6ed0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/ProcessingTime.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.windowing.time;
+
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The definition of a processing time interval for windowing. See
+ * {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime} for a definition
+ * of processing time.
+ */
+public final class ProcessingTime extends AbstractTime {
+
+	/** Instantiation only via factory method */
+	private ProcessingTime(long size, TimeUnit unit) {
+		super(size, unit);
+	}
+
+	@Override
+	public ProcessingTime makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic characteristic) {
+		if (characteristic == TimeCharacteristic.ProcessingTime) {
+			return this;
+		}
+		else {
+			throw new InvalidProgramException(
+					"Cannot use ProcessingTime policy in a dataflow that runs on " + characteristic);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Factory
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a processing time policy describing a processing time interval.
+	 *
+	 * @param size The size of the generated windows.
+	 * @param unit The init (seconds, milliseconds) of the time interval.
+	 * @return The processing time policy.
+	 */
+	public static ProcessingTime of(long size, TimeUnit unit) {
+		return new ProcessingTime(size, unit);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
new file mode 100644
index 0000000..d1b3fe3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.windowing.time;
+
+import org.apache.flink.streaming.api.TimeCharacteristic;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The definition of a time interval for windowing. The time characteristic referred
+ * to is the default time characteristic set on the execution environment.
+ */
+public final class Time extends AbstractTime {
+
+	/** Instantiation only via factory method */
+	private Time(long size, TimeUnit unit) {
+		super(size, unit);
+	}
+
+	@Override
+	public AbstractTime makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic timeCharacteristic) {
+		switch (timeCharacteristic) {
+			case ProcessingTime:
+				return ProcessingTime.of(getSize(), getUnit());
+			case IngestionTime:
+			case EventTime:
+				return EventTime.of(getSize(), getUnit());
+			default:
+				throw new IllegalArgumentException("Unknown time characteristic");
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Factory
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a time policy describing a processing time interval. The policy refers to the
+	 * time characteristic that is set on the dataflow via
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#
+	 * setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}.
+	 *
+	 * @param size The size of the generated windows.
+	 * @param unit The init (seconds, milliseconds) of the time interval.
+	 * @return The time policy.
+	 */
+	public static Time of(long size, TimeUnit unit) {
+		return new Time(size, unit);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
index 64850a2..da198be 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.triggers;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
 public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigger<Object, W> {
@@ -68,6 +69,11 @@ public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigge
 		return new ContinuousProcessingTimeTrigger<>(granularity);
 	}
 
+	@VisibleForTesting
+	public long getGranularity() {
+		return granularity;
+	}
+
 	@Override
 	public String toString() {
 		return "ContinuousProcessingTimeTrigger(" + granularity + ")";

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
index b7f085a..3b6dc6d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousWatermarkTrigger.java
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.streaming.api.windowing.triggers;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 
 public class ContinuousWatermarkTrigger<W extends Window> implements Trigger<Object, W> {
@@ -59,6 +60,11 @@ public class ContinuousWatermarkTrigger<W extends Window> implements Trigger<Obj
 		return "ContinuousProcessingTimeTrigger(" + granularity + ")";
 	}
 
+	@VisibleForTesting
+	public long getGranularity() {
+		return granularity;
+	}
+
 	public static <W extends Window> ContinuousWatermarkTrigger<W> of(long granularity) {
 		return new ContinuousWatermarkTrigger<>(granularity);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/AbstractTimePolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/AbstractTimePolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/AbstractTimePolicy.java
deleted file mode 100644
index 6e382bd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/AbstractTimePolicy.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.windowpolicy;
-
-import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-public class AbstractTimePolicy extends WindowPolicy {
-
-	private static final long serialVersionUID = 6593098375698927728L;
-	
-	/** The time unit for this policy's time interval */
-	private final TimeUnit unit;
-	
-	/** The size of the windows generated by this policy */
-	private final long size;
-
-
-	protected AbstractTimePolicy(long size, TimeUnit unit) {
-		this.unit = checkNotNull(unit, "time unit may not be null");
-		this.size = size;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the time unit for this policy's time interval.
-	 * @return The time unit for this policy's time interval.
-	 */
-	public TimeUnit getUnit() {
-		return unit;
-	}
-
-	/**
-	 * Gets the length of this policy's time interval.
-	 * @return The length of this policy's time interval.
-	 */
-	public long getSize() {
-		return size;
-	}
-
-	/**
-	 * Converts the time interval to milliseconds.
-	 * @return The time interval in milliseconds.
-	 */
-	public long toMilliseconds() {
-		return unit.toMillis(size);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String toString(WindowPolicy slidePolicy) {
-		if (slidePolicy == null) {
-			return "Tumbling Window (" + getClass().getSimpleName() + ") (" + size + ' ' + unit.name() + ')';
-		}
-		else if (slidePolicy.getClass() == getClass()) {
-			AbstractTimePolicy timeSlide = (AbstractTimePolicy) slidePolicy;
-			
-			return "Sliding Window (" + getClass().getSimpleName() + ") (length="
-					+ size + ' ' + unit.name() + ", slide=" + timeSlide.size + ' ' + timeSlide.unit.name() + ')';
-		}
-		else {
-			return super.toString(slidePolicy);
-		}
-	}
-	
-	@Override
-	public int hashCode() {
-		return 31 * (int) (size ^ (size >>> 32)) + unit.hashCode();
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj != null && obj.getClass() == getClass()) {
-			AbstractTimePolicy that = (AbstractTimePolicy) obj;
-			return this.size == that.size && this.unit.equals(that.unit);
-		}
-		else {
-			return false;
-		}
-	}
-
-	@Override
-	public String toString() {
-		return getClass().getSimpleName() + " (" + size + ' ' + unit.name() + ')';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Count.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Count.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Count.java
deleted file mode 100644
index 5fb7d58..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Count.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.windowpolicy;
-
-
-/**
- * A windowing policy that generates windows based on element counts.
- */
-public final class Count extends WindowPolicy {
-
-	private static final long serialVersionUID = 3197290738634320211L;
-
-	private long size;
-
-	/** Instantiation only via factory method */
-	private Count(long size) {
-		this.size = size;
-	}
-
-	public long getSize() {
-		return size;
-	}
-
-	@Override
-	public String toString() {
-		return "Count Window (" + size + ')';
-	}
-
-	// ------------------------------------------------------------------------
-	//  Factory
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a count based windowing policy
-	 *
-	 * @param size The size of the generated windows.
-	 * @return The time policy.
-	 */
-	public static Count of(long size) {
-		return new Count(size);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Delta.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Delta.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Delta.java
deleted file mode 100644
index 4a3082c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Delta.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.windowpolicy;
-
-
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-
-/**
- * A windowing policy that generates windows based on a delta between elements.
- */
-public final class Delta<T> extends WindowPolicy {
-
-	private static final long serialVersionUID = 3197290738634320211L;
-
-	private DeltaFunction<T> deltaFunction;
-
-	private double threshold;
-
-	/** Instantiation only via factory method */
-	private Delta(DeltaFunction<T> deltaFunction, double threshold) {
-		this.deltaFunction = deltaFunction;
-		this.threshold = threshold;
-	}
-
-	public DeltaFunction<T> getDeltaFunction() {
-		return deltaFunction;
-	}
-
-	public double getThreshold() {
-		return threshold;
-	}
-
-	@Override
-	public String toString() {
-		return "Delta Window (" + deltaFunction + ", " + threshold + ')';
-	}
-
-	// ------------------------------------------------------------------------
-	//  Factory
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a delta based windowing policy
-	 *
-	 * @param threshold The threshold for deltas at which to trigger windows
-	 * @param deltaFunction The delta function
-	 * @return The time policy.
-	 */
-	public static <T> Delta<T> of(double threshold, DeltaFunction<T> deltaFunction) {
-		return new Delta<T>(deltaFunction, threshold);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/EventTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/EventTime.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/EventTime.java
deleted file mode 100644
index c32a0b0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/EventTime.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.windowpolicy;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * The definition of an event time interval for windowing. See
- * {@link org.apache.flink.streaming.api.TimeCharacteristic#EventTime} for a definition
- * of event time.
- */
-public final class EventTime extends AbstractTimePolicy {
-
-	private static final long serialVersionUID = 8333566691833596747L;
-
-	/** Instantiation only via factory method */
-	private EventTime(long size, TimeUnit unit) {
-		super(size, unit);
-	}
-
-	@Override
-	public EventTime makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic characteristic) {
-		if (characteristic == TimeCharacteristic.EventTime || characteristic == TimeCharacteristic.IngestionTime) {
-			return this;
-		}
-		else {
-			throw new InvalidProgramException(
-					"Cannot use EventTime policy in a dataflow that runs on " + characteristic);
-		}
-	}
-	// ------------------------------------------------------------------------
-	//  Factory
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates an event time policy describing an event time interval.
-	 *
-	 * @param size The size of the generated windows.
-	 * @param unit The init (seconds, milliseconds) of the time interval.
-	 * @return The event time policy.
-	 */
-	public static EventTime of(long size, TimeUnit unit) {
-		return new EventTime(size, unit);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/ProcessingTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/ProcessingTime.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/ProcessingTime.java
deleted file mode 100644
index a71ba1d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/ProcessingTime.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.windowpolicy;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * The definition of a processing time interval for windowing. See
- * {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime} for a definition
- * of processing time.
- */
-public final class ProcessingTime extends AbstractTimePolicy {
-
-	private static final long serialVersionUID = 7546166721132583007L;
-
-	/** Instantiation only via factory method */
-	private ProcessingTime(long size, TimeUnit unit) {
-		super(size, unit);
-	}
-
-	@Override
-	public ProcessingTime makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic characteristic) {
-		if (characteristic == TimeCharacteristic.ProcessingTime) {
-			return this;
-		}
-		else {
-			throw new InvalidProgramException(
-					"Cannot use ProcessingTime policy in a dataflow that runs on " + characteristic);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Factory
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a processing time policy describing a processing time interval.
-	 *
-	 * @param size The size of the generated windows.
-	 * @param unit The init (seconds, milliseconds) of the time interval.
-	 * @return The processing time policy.
-	 */
-	public static ProcessingTime of(long size, TimeUnit unit) {
-		return new ProcessingTime(size, unit);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Time.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Time.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Time.java
deleted file mode 100644
index efc9bf2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/Time.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.windowpolicy;
-
-import org.apache.flink.streaming.api.TimeCharacteristic;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * The definition of a time interval for windowing. The time characteristic referred
- * to is the default time characteristic set on the execution environment.
- */
-public final class Time extends AbstractTimePolicy {
-
-	private static final long serialVersionUID = 3197290738634320211L;
-
-	/** Instantiation only via factory method */
-	private Time(long size, TimeUnit unit) {
-		super(size, unit);
-	}
-
-	@Override
-	public AbstractTimePolicy makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic timeCharacteristic) {
-		switch (timeCharacteristic) {
-			case ProcessingTime:
-				return ProcessingTime.of(getSize(), getUnit());
-			case IngestionTime:
-			case EventTime:
-				return EventTime.of(getSize(), getUnit());
-			default:
-				throw new IllegalArgumentException("Unknown time characteristic");
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Factory
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a time policy describing a processing time interval. The policy refers to the
-	 * time characteristic that is set on the dataflow via
-	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#
-	 * setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}.
-	 *
-	 * @param size The size of the generated windows.
-	 * @param unit The init (seconds, milliseconds) of the time interval.
-	 * @return The time policy.
-	 */
-	public static Time of(long size, TimeUnit unit) {
-		return new Time(size, unit);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/WindowPolicy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/WindowPolicy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/WindowPolicy.java
deleted file mode 100644
index 2e1a387..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windowpolicy/WindowPolicy.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.windowpolicy;
-
-import org.apache.flink.streaming.api.TimeCharacteristic;
-
-/**
- * The base class of all window policies. Window policies define how windows
- * are formed over the data stream.
- */
-public abstract class WindowPolicy implements java.io.Serializable {
-
-	private static final long serialVersionUID = -8696529489282723113L;
-	
-	/**
-	 * If the concrete instantiation of a window policy depends on the time characteristic of the
-	 * dataflow (processing time, event time), then this method must be overridden to convert this
-	 * policy to the respective specific instantiation.
-	 * <p>
-	 * The {@link Time} policy for example, will convert itself to an {@link ProcessingTime} policy,
-	 * if the time characteristic is set to {@link TimeCharacteristic#ProcessingTime}.
-	 * <p>
-	 * By default, this method does nothing and simply returns this object itself.
-	 * 
-	 * @param characteristic The time characteristic of the dataflow.
-	 * @return The specific instantiation of this policy, or the policy itself. 
-	 */
-	public WindowPolicy makeSpecificBasedOnTimeCharacteristic(TimeCharacteristic characteristic) {
-		return this;
-	}
-	
-	
-	public String toString(WindowPolicy slidePolicy) {
-		if (slidePolicy != null) {
-			return "Window [" + toString() + ", slide=" + slidePolicy + ']';
-		}
-		else {
-			return "Window [" + toString() + ']';
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyToOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyToOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyToOperator.java
deleted file mode 100644
index 880c85c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyToOperator.java
+++ /dev/null
@@ -1,239 +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.runtime.operators.windowing;
-
-import org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
-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.functions.windowing.ReduceWindowFunction;
-import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
-import org.apache.flink.streaming.api.windowing.evictors.DeltaEvictor;
-import org.apache.flink.streaming.api.windowing.evictors.Evictor;
-import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
-import org.apache.flink.streaming.api.windowing.triggers.ContinuousProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ContinuousWatermarkTrigger;
-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.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.triggers.WatermarkTrigger;
-import org.apache.flink.streaming.api.windowing.windowpolicy.Count;
-import org.apache.flink.streaming.api.windowing.windowpolicy.Delta;
-import org.apache.flink.streaming.api.windowing.windowpolicy.EventTime;
-import org.apache.flink.streaming.api.windowing.windowpolicy.ProcessingTime;
-import org.apache.flink.streaming.api.windowing.windowpolicy.WindowPolicy;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.EvictingWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
-
-/**
- * This class implements the conversion from window policies to concrete operator
- * implementations.
- */
-public class PolicyToOperator {
-
-	/**
-	 * Entry point to create an operator for the given window policies and the window function.
-	 */
-	public static <IN, OUT, KEY> OneInputStreamOperator<IN, OUT> createOperatorForPolicies(
-			WindowPolicy window, WindowPolicy slide, Function function, KeySelector<IN, KEY> keySelector)
-	{
-		if (window == null || function == null) {
-			throw new NullPointerException();
-		}
-		
-		// -- case 1: both policies are processing time policies
-		if (window instanceof ProcessingTime && (slide == null || slide instanceof ProcessingTime)) {
-			final long windowLength = ((ProcessingTime) window).toMilliseconds();
-			final long windowSlide = slide == null ? windowLength : ((ProcessingTime) slide).toMilliseconds();
-			
-			if (function instanceof ReduceFunction) {
-				@SuppressWarnings("unchecked")
-				ReduceFunction<IN> reducer = (ReduceFunction<IN>) function;
-
-				@SuppressWarnings("unchecked")
-				OneInputStreamOperator<IN, OUT> op = (OneInputStreamOperator<IN, OUT>)
-						new AggregatingProcessingTimeWindowOperator<>(
-								reducer, keySelector, windowLength, windowSlide);
-				return op;
-			}
-			else if (function instanceof KeyedWindowFunction) {
-				@SuppressWarnings("unchecked")
-				KeyedWindowFunction<IN, OUT, KEY, Window> wf = (KeyedWindowFunction<IN, OUT, KEY, Window>) function;
-
-				return new AccumulatingProcessingTimeWindowOperator<>(
-								wf, keySelector, windowLength, windowSlide);
-			}
-		}
-
-		// -- case 2: both policies are event time policies
-		if (window instanceof EventTime && (slide == null || slide instanceof EventTime)) {
-			final long windowLength = ((EventTime) window).toMilliseconds();
-			final long windowSlide = slide == null ? windowLength : ((EventTime) slide).toMilliseconds();
-
-			WindowAssigner<? super IN, TimeWindow> assigner;
-			if (windowSlide == windowLength) {
-				assigner = TumblingTimeWindows.of(windowLength);
-			} else {
-				assigner = SlidingTimeWindows.of(windowLength, windowSlide);
-			}
-			WindowBufferFactory<IN, ? extends WindowBuffer<IN>> windowBuffer;
-			if (function instanceof ReduceFunction) {
-				@SuppressWarnings("unchecked")
-				ReduceFunction<IN> reducer = (ReduceFunction<IN>) SerializationUtils.clone(function);
-				function = new ReduceWindowFunction<>(reducer);
-				windowBuffer = new PreAggregatingHeapWindowBuffer.Factory<>(reducer);
-			} else {
-				windowBuffer = new HeapWindowBuffer.Factory<>();
-			}
-			@SuppressWarnings("unchecked")
-			KeyedWindowFunction<IN, OUT, KEY, TimeWindow> windowFunction = (KeyedWindowFunction<IN, OUT, KEY, TimeWindow>) function;
-
-			return new WindowOperator<>(
-					assigner,
-					keySelector,
-					windowBuffer,
-					windowFunction,
-					WatermarkTrigger.create());
-		}
-
-		// -- case 3: arbitrary trigger, no eviction
-		if (slide == null) {
-			Trigger<? super IN, GlobalWindow> trigger = policyToTrigger(window);
-			// we need to make them purging triggers because the trigger/eviction policy model
-			// expects that the window is purged when no slide is used
-			Trigger<? super IN, GlobalWindow> purgingTrigger = PurgingTrigger.of(trigger);
-
-			WindowBufferFactory<IN, ? extends WindowBuffer<IN>> windowBuffer;
-			if (function instanceof ReduceFunction) {
-				@SuppressWarnings("unchecked")
-				ReduceFunction<IN> reducer = (ReduceFunction<IN>) SerializationUtils.clone(function);
-				function = new ReduceWindowFunction<>(reducer);
-				windowBuffer = new PreAggregatingHeapWindowBuffer.Factory<>(reducer);
-			} else {
-				windowBuffer = new HeapWindowBuffer.Factory<>();
-			}
-
-			if (!(function instanceof KeyedWindowFunction)) {
-				throw new IllegalStateException("Windowing function is not of type EvaluateKeyedWindowFunction.");
-			}
-			@SuppressWarnings("unchecked")
-			KeyedWindowFunction<IN, OUT, KEY, GlobalWindow> windowFunction = (KeyedWindowFunction<IN, OUT, KEY, GlobalWindow>) function;
-
-			return new WindowOperator<>(
-					GlobalWindows.<IN>create(),
-					keySelector,
-					windowBuffer,
-					windowFunction,
-					purgingTrigger);
-		}
-
-		// -- case 4: arbitrary trigger, arbitrary eviction
-		Trigger<? super IN, GlobalWindow> trigger = policyToTrigger(slide);
-		Evictor<? super IN, GlobalWindow> evictor = policyToEvictor(window);
-
-		WindowBufferFactory<IN, ? extends EvictingWindowBuffer<IN>> windowBuffer = new HeapWindowBuffer.Factory<>();
-		if (function instanceof ReduceFunction) {
-			@SuppressWarnings("unchecked")
-			ReduceFunction<IN> reducer = (ReduceFunction<IN>) SerializationUtils.clone(function);
-			function = new ReduceWindowFunction<>(reducer);
-		}
-
-		if (!(function instanceof KeyedWindowFunction)) {
-			throw new IllegalStateException("Windowing function is not of type EvaluateKeyedWindowFunction.");
-		}
-
-		@SuppressWarnings("unchecked")
-		KeyedWindowFunction<IN, OUT, KEY, GlobalWindow> windowFunction = (KeyedWindowFunction<IN, OUT, KEY, GlobalWindow>) function;
-
-		EvictingWindowOperator<KEY, IN, OUT, GlobalWindow> op = new EvictingWindowOperator<>(
-				GlobalWindows.<IN>create(),
-				keySelector,
-				windowBuffer,
-				windowFunction,
-				trigger,
-				evictor);
-
-		if (window instanceof ProcessingTime) {
-			// special case, we need to instruct the window operator to store the processing time in
-			// the elements so that the evictor can work on that
-			op.enableSetProcessingTime(true);
-		}
-
-		return op;
-	}
-
-	private static <IN> Trigger<? super IN, GlobalWindow> policyToTrigger(WindowPolicy policy) {
-		if (policy instanceof EventTime) {
-			EventTime eventTime = (EventTime) policy;
-			return ContinuousWatermarkTrigger.of(eventTime.getSize());
-		} else if (policy instanceof ProcessingTime) {
-			ProcessingTime processingTime = (ProcessingTime) policy;
-			return ContinuousProcessingTimeTrigger.of(processingTime.getSize());
-		} else if (policy instanceof Count) {
-			Count count = (Count) policy;
-			return CountTrigger.of(count.getSize());
-		} else if (policy instanceof Delta) {
-			@SuppressWarnings("unchecked,rawtypes")
-			Delta<IN> delta = (Delta) policy;
-			return DeltaTrigger.of(delta.getThreshold(), delta.getDeltaFunction());
-
-		}
-
-		throw new UnsupportedOperationException("Unsupported policy " + policy);
-	}
-
-	private static <IN> Evictor<? super IN, GlobalWindow> policyToEvictor(WindowPolicy policy) {
-		if (policy instanceof EventTime) {
-			EventTime eventTime = (EventTime) policy;
-			return TimeEvictor.of(eventTime.getSize());
-		} else if (policy instanceof ProcessingTime) {
-			ProcessingTime processingTime = (ProcessingTime) policy;
-			return TimeEvictor.of(processingTime.getSize());
-		} else if (policy instanceof Count) {
-			Count count = (Count) policy;
-			return CountEvictor.of(count.getSize());
-		} else if (policy instanceof Delta) {
-			@SuppressWarnings("unchecked,rawtypes")
-			Delta<IN> delta = (Delta) policy;
-			return DeltaEvictor.of(delta.getThreshold(), delta.getDeltaFunction());
-
-		}
-
-
-		throw new UnsupportedOperationException("Unsupported policy " + policy);
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	/** Don't instantiate */
-	private PolicyToOperator() {}
-}


[7/9] flink git commit: [FLINK-2561] [gelly] add Scala Gelly docs

Posted by se...@apache.org.
[FLINK-2561] [gelly] add Scala Gelly docs

This closes #1204


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

Branch: refs/heads/master
Commit: 937793e17a177e6165f8726b4f0de3b7fa197e45
Parents: f2ea4e4
Author: vasia <va...@apache.org>
Authored: Wed Sep 30 17:53:10 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 1 11:04:00 2015 +0200

----------------------------------------------------------------------
 docs/libs/gelly_guide.md | 653 +++++++++++++++++++++++++++++++++++++++---
 1 file changed, 618 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/937793e1/docs/libs/gelly_guide.md
----------------------------------------------------------------------
diff --git a/docs/libs/gelly_guide.md b/docs/libs/gelly_guide.md
index b6a0533..0c3748b 100644
--- a/docs/libs/gelly_guide.md
+++ b/docs/libs/gelly_guide.md
@@ -23,7 +23,7 @@ under the License.
 
 <a href="#top"></a>
 
-Gelly is a Java Graph API for Flink. It contains a set of methods and utilities which aim to simplify the development of graph analysis applications in Flink. In Gelly, graphs can be transformed and modified using high-level functions similar to the ones provided by the batch processing API. Gelly provides methods to create, transform and modify graphs, as well as a library of graph algorithms.
+Gelly is a Graph API for Flink. It contains a set of methods and utilities which aim to simplify the development of graph analysis applications in Flink. In Gelly, graphs can be transformed and modified using high-level functions similar to the ones provided by the batch processing API. Gelly provides methods to create, transform and modify graphs, as well as a library of graph algorithms.
 
 * This will be replaced by the TOC
 {:toc}
@@ -35,17 +35,30 @@ Gelly is currently part of the *staging* Maven project. All relevant classes are
 
 Add the following dependency to your `pom.xml` to use Gelly.
 
-~~~xml
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
+{% highlight xml %}
 <dependency>
     <groupId>org.apache.flink</groupId>
     <artifactId>flink-gelly</artifactId>
     <version>{{site.version}}</version>
 </dependency>
-~~~
+{% endhighlight %}
+</div>
+<div data-lang="scala" markdown="1">
+{% highlight xml %}
+<dependency>
+    <groupId>org.apache.flink</groupId>
+    <artifactId>flink-gelly-scala</artifactId>
+    <version>{{site.version}}</version>
+</dependency>
+{% endhighlight %}
+</div>
+</div>
 
 Note that Gelly is currently not part of the binary distribution. See linking with it for cluster execution [here](../apis/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution).
 
-The remaining sections provide a description of available methods and present several examples of how to use Gelly and how to mix it with the Flink Java API. After reading this guide, you might also want to check the {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ "Gelly examples" %}.
+The remaining sections provide a description of available methods and present several examples of how to use Gelly and how to mix it with the Flink DataSet API. After reading this guide, you might also want to check the {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/ "Gelly examples" %}.
 
 Graph Representation
 -----------
@@ -54,6 +67,8 @@ In Gelly, a `Graph` is represented by a `DataSet` of vertices and a `DataSet` of
 
 The `Graph` nodes are represented by the `Vertex` type. A `Vertex` is defined by a unique ID and a value. `Vertex` IDs should implement the `Comparable` interface. Vertices without value can be represented by setting the value type to `NullValue`.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 // create a new vertex with a Long ID and a String value
 Vertex<Long, String> v = new Vertex<Long, String>(1L, "foo");
@@ -61,9 +76,23 @@ Vertex<Long, String> v = new Vertex<Long, String>(1L, "foo");
 // create a new vertex with a Long ID and no value
 Vertex<Long, NullValue> v = new Vertex<Long, NullValue>(1L, NullValue.getInstance());
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+// create a new vertex with a Long ID and a String value
+val v = new Vertex(1L, "foo")
+
+// create a new vertex with a Long ID and no value
+val v = new Vertex(1L, NullValue.getInstance())
+{% endhighlight %}
+</div>
+</div>
 
 The graph edges are represented by the `Edge` type. An `Edge` is defined by a source ID (the ID of the source `Vertex`), a target ID (the ID of the target `Vertex`) and an optional value. The source and target IDs should be of the same type as the `Vertex` IDs. Edges with no value have a `NullValue` value type.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 Edge<Long, Double> e = new Edge<Long, Double>(1L, 2L, 0.5);
 
@@ -72,6 +101,19 @@ Edge<Long, Double> reversed = e.reverse();
 
 Double weight = e.getValue(); // weight = 0.5
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val e = new Edge(1L, 2L, 0.5)
+
+// reverse the source and target of this edge
+val reversed = e.reverse
+
+val weight = e.getValue // weight = 0.5
+{% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
@@ -82,6 +124,8 @@ You can create a `Graph` in the following ways:
 
 * from a `DataSet` of edges and an optional `DataSet` of vertices:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -91,9 +135,25 @@ DataSet<Edge<String, Double>> edges = ...
 
 Graph<String, Long, Double> graph = Graph.fromDataSet(vertices, edges, env);
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+val vertices: DataSet[Vertex[String, Long]] = ...
+
+val edges: DataSet[Edge[String, Double]] = ...
+
+val graph = Graph.fromDataSet(vertices, edges, env)
+{% endhighlight %}
+</div>
+</div>
 
 * from a `DataSet` of `Tuple3` and an optional `DataSet` of `Tuple2`. In this case, Gelly will convert each `Tuple3` to an `Edge`, where the first field will be the source ID, the second field will be the target ID and the third field will be the edge value. Equivalently, each `Tuple2` will be converted to a `Vertex`, where the first field will be the vertex ID and the second field will be the vertex value:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -111,7 +171,6 @@ Graph<String, Long, Double> graph = Graph.fromTupleDataSet(vertexTuples, edgeTup
 - `vertexTypes(Class<K> vertexKey, Class<VV> vertexValue)`: the Graph has vertex values, but no edge values.
 - `keyType(Class<K> vertexKey)`: the Graph has no vertex values and no edge values.
 
-
 {% highlight java %}
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -123,10 +182,26 @@ Graph<String, Long, Double> graph = Graph.fromCsvReader("path/to/vertex/input",
 // create a Graph with no Vertex or Edge values
 Graph<Long, NullValue, NullValue> simpleGraph = Graph.fromCsvReader("path/to/edge/input", env).keyType(Long.class);
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+val vertexTuples = env.readCsvFile[String, Long]("path/to/vertex/input")
+
+val edgeTuples = env.readCsvFile[String, String, Double]("path/to/edge/input")
+
+val graph = Graph.fromTupleDataSet(vertexTuples, edgeTuples, env)
+{% endhighlight %}
+</div>
+</div>
 
 
 * from a `Collection` of edges and an optional `Collection` of vertices:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -143,13 +218,39 @@ If no vertex input is provided during Graph creation, Gelly will automatically p
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
 // initialize the vertex value to be equal to the vertex ID
-Graph<Long, Long, String> graph = Graph.fromCollection(edges, 
+Graph<Long, Long, String> graph = Graph.fromCollection(edgeList, 
 				new MapFunction<Long, Long>() {
 					public Long map(Long value) { 
 						return value; 
 					} 
 				}, env);
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+val vertexList = List(...)
+
+val edgeList = List(...)
+
+val graph = Graph.fromCollection(vertexList, edgeList, env)
+{% endhighlight %}
+
+If no vertex input is provided during Graph creation, Gelly will automatically produce the `Vertex` `DataSet` from the edge input. In this case, the created vertices will have no values. Alternatively, you can provide a `MapFunction` as an argument to the creation method, in order to initialize the `Vertex` values:
+
+{% highlight java %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+// initialize the vertex value to be equal to the vertex ID
+val graph = Graph.fromCollection(edgeList, env,
+    new MapFunction[Long, Long] {
+       def map(id: Long): Long = id
+    })
+{% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
@@ -158,6 +259,8 @@ Graph Properties
 
 Gelly includes the following methods for retrieving various Graph properties and metrics:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 // get the Vertex DataSet
 DataSet<Vertex<K, VV>> getVertices()
@@ -190,6 +293,43 @@ long numberOfEdges()
 DataSet<Triplet<K, VV, EV>> getTriplets()
 
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+// get the Vertex DataSet
+getVertices: DataSet[Vertex[K, VV]]
+
+// get the Edge DataSet
+getEdges: DataSet[Edge[K, EV]]
+
+// get the IDs of the vertices as a DataSet
+getVertexIds: DataSet[K]
+
+// get the source-target pairs of the edge IDs as a DataSet
+getEdgeIds: DataSet[(K, K)]
+
+// get a DataSet of <vertex ID, in-degree> pairs for all vertices
+inDegrees: DataSet[(K, Long)]
+
+// get a DataSet of <vertex ID, out-degree> pairs for all vertices
+outDegrees: DataSet[(K, Long)]
+
+// get a DataSet of <vertex ID, degree> pairs for all vertices, where degree is the sum of in- and out- degrees
+getDegrees: DataSet[(K, Long)]
+
+// get the number of vertices
+numberOfVertices: Long
+
+// get the number of edges
+numberOfEdges: Long
+
+// get a DataSet of Triplets<srcVertex, trgVertex, edge>
+getTriplets: DataSet[Triplet[K, VV, EV]]
+
+{% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
@@ -198,6 +338,8 @@ Graph Transformations
 
 * <strong>Map</strong>: Gelly provides specialized methods for applying a map transformation on the vertex values or edge values. `mapVertices` and `mapEdges` return a new `Graph`, where the IDs of the vertices (or edges) remain unchanged, while the values are transformed according to the provided user-defined map function. The map functions also allow changing the type of the vertex or edge values.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 Graph<Long, Long, Long> graph = Graph.fromDataSet(vertices, edges, env);
@@ -210,9 +352,23 @@ Graph<Long, Long, Long> updatedGraph = graph.mapVertices(
 					}
 				});
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+val graph = Graph.fromDataSet(vertices, edges, env)
+
+// increment each vertex value by one
+val updatedGraph = graph.mapVertices(v => v.getValue + 1)
+{% endhighlight %}
+</div>
+</div>
 
 * <strong>Filter</strong>: A filter transformation applies a user-defined filter function on the vertices or edges of the `Graph`. `filterOnEdges` will create a sub-graph of the original graph, keeping only the edges that satisfy the provided predicate. Note that the vertex dataset will not be modified. Respectively, `filterOnVertices` applies a filter on the vertices of the graph. Edges whose source and/or target do not satisfy the vertex predicate are removed from the resulting edge dataset. The `subgraph` method can be used to apply a filter function to the vertices and the edges at the same time.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 Graph<Long, Long, Long> graph = ...
 
@@ -230,6 +386,18 @@ graph.subgraph(
 				}
 		})
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val graph: Graph[Long, Long, Long] = ...
+
+// keep only vertices with positive values
+// and only edges with negative values
+graph.subgraph((vertex => vertex.getValue > 0), (edge => edge.getValue < 0))
+{% endhighlight %}
+</div>
+</div>
 
 <p class="text-center">
     <img alt="Filter Transformations" width="80%" src="fig/gelly-filter.png"/>
@@ -239,6 +407,8 @@ graph.subgraph(
 Similarly, an input dataset can be joined with the edges, using one of three methods. `joinWithEdges` expects an input `DataSet` of `Tuple3` and joins on the composite key of both source and target vertex IDs. `joinWithEdgesOnSource` expects a `DataSet` of `Tuple2` and joins on the source key of the edges and the first attribute of the input dataset and `joinWithEdgesOnTarget` expects a `DataSet` of `Tuple2` and joins on the target key of the edges and the first attribute of the input dataset. All three methods apply a map function on the edge and the input data set values.
 Note that if the input dataset contains a key multiple times, all Gelly join methods will only consider the first value encountered.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 Graph<Long, Double, Double> network = ...
 
@@ -252,6 +422,19 @@ Graph<Long, Double, Double> networkWithWeights = network.joinWithEdgesOnSource(v
 					}
 				});
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val network: Graph[Long, Double, Double] = ...
+
+val vertexOutDegrees: DataSet[(Long, Long)] = network.outDegrees
+
+// assign the transition probabilities as the edge weights
+val networkWithWeights = network.joinWithEdgesOnSource(vertexOutDegrees, (v1: Double, v2: Long) => v1 / v2)
+{% endhighlight %}
+</div>
+</div>
 
 * <strong>Reverse</strong>: the `reverse()` method returns a new `Graph` where the direction of all edges has been reversed.
 
@@ -272,6 +455,8 @@ Graph Mutations
 
 Gelly includes the following methods for adding and removing vertices and edges from an input `Graph`:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 // adds a Vertex to the Graph. If the Vertex already exists, it will not be added again.
 Graph<K, VV, EV> addVertex(final Vertex<K, VV> vertex)
@@ -296,9 +481,37 @@ Graph<K, VV, EV> removeEdge(Edge<K, EV> edge)
 
 // removes *all* edges that match the edges in the given list
 Graph<K, VV, EV> removeEdges(List<Edge<K, EV>> edgesToBeRemoved)
+{% endhighlight %}
+</div>
 
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+// adds a Vertex to the Graph. If the Vertex already exists, it will not be added again.
+addVertex(vertex: Vertex[K, VV])
 
+// adds a list of vertices to the Graph. If the vertices already exist in the graph, they will not be added once more.
+addVertices(verticesToAdd: List[Vertex[K, VV]])
+
+// adds an Edge to the Graph. If the source and target vertices do not exist in the graph, they will also be added.
+addEdge(source: Vertex[K, VV], target: Vertex[K, VV], edgeValue: EV)
+
+// adds a list of edges to the Graph. When adding an edge for a non-existing set of vertices, the edge is considered invalid and ignored.
+addEdges(edges: List[Edge[K, EV]])
+
+// removes the given Vertex and its edges from the Graph.
+removeVertex(vertex: Vertex[K, VV])
+
+// removes the given list of vertices and their edges from the Graph
+removeVertices(verticesToBeRemoved: List[Vertex[K, VV]])
+
+// removes *all* edges that match the given Edge from the Graph.
+removeEdge(edge: Edge[K, EV])
+
+// removes *all* edges that match the edges in the given list
+removeEdges(edgesToBeRemoved: List[Edge[K, EV]])
 {% endhighlight %}
+</div>
+</div>
 
 Neighborhood Methods
 -----------
@@ -315,6 +528,8 @@ For example, assume that you want to select the minimum weight of all out-edges
 
 The following code will collect the out-edges for each vertex and apply the `SelectMinWeight()` user-defined function on each of the resulting neighborhoods:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 Graph<Long, Long, Double> graph = ...
 
@@ -329,6 +544,23 @@ static final class SelectMinWeight implements ReduceEdgesFunction<Double> {
 		}
 }
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val graph: Graph[Long, Long, Double] = ...
+
+val minWeights = graph.reduceOnEdges(new SelectMinWeight, EdgeDirection.OUT)
+
+// user-defined function to select the minimum weight
+final class SelectMinWeight extends ReduceEdgesFunction[Double] {
+	override def reduceEdges(firstEdgeValue: Double, secondEdgeValue: Double): Double = {
+		Math.min(firstEdgeValue, secondEdgeValue)
+	}
+ }
+{% endhighlight %}
+</div>
+</div>
 
 <p class="text-center">
     <img alt="reduceOnEdges Example" width="50%" src="fig/gelly-reduceOnEdges.png"/>
@@ -336,6 +568,8 @@ static final class SelectMinWeight implements ReduceEdgesFunction<Double> {
 
 Similarly, assume that you would like to compute the sum of the values of all in-coming neighbors, for every vertex. The following code will collect the in-coming neighbors for each vertex and apply the `SumValues()` user-defined function on each neighborhood:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 Graph<Long, Long, Double> graph = ...
 
@@ -350,6 +584,23 @@ static final class SumValues implements ReduceNeighborsFunction<Long> {
 	  	}
 }
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val graph: Graph[Long, Long, Double] = ...
+
+val verticesWithSum = graph.reduceOnNeighbors(new SumValues, EdgeDirection.IN)
+
+// user-defined function to sum the neighbor values
+final class SumValues extends ReduceNeighborsFunction[Long] {
+   	override def reduceNeighbors(firstNeighbor: Long, secondNeighbor: Long): Long = {
+    	firstNeighbor + secondNeighbor
+    }
+}
+{% endhighlight %}
+</div>
+</div>
 
 <p class="text-center">
     <img alt="reduceOnNeighbors Example" width="70%" src="fig/gelly-reduceOnNeighbors.png"/>
@@ -361,6 +612,8 @@ These methods return zero, one or more values per vertex and provide access to t
 
 For example, the following code will output all the vertex pairs which are connected with an edge having a weight of 0.5 or more:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 Graph<Long, Long, Double> graph = ...
 
@@ -383,6 +636,32 @@ static final class SelectLargeWeightNeighbors implements NeighborsFunctionWithVe
 		}
 }
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val graph: Graph[Long, Long, Double] = ...
+
+val vertexPairs = graph.groupReduceOnNeighbors(new SelectLargeWeightNeighbors, EdgeDirection.OUT)
+
+// user-defined function to select the neighbors which have edges with weight > 0.5
+final class SelectLargeWeightNeighbors extends NeighborsFunctionWithVertexValue[Long, Long, Double, 
+  (Vertex[Long, Long], Vertex[Long, Long])] {
+
+	override def iterateNeighbors(vertex: Vertex[Long, Long],
+		neighbors: Iterable[(Edge[Long, Double], Vertex[Long, Long])],
+		out: Collector[(Vertex[Long, Long], Vertex[Long, Long])]) = {
+
+			for (neighbor <- neighbors) {
+				if (neighbor._1.getValue() > 0.5) {
+					out.collect(vertex, neighbor._2);
+				}
+			}
+		}
+   }
+{% endhighlight %}
+</div>
+</div>
 
 When the aggregation computation does not require access to the vertex value (for which the aggregation is performed), it is advised to use the more efficient `EdgesFunction` and `NeighborsFunction` for the user-defined functions. When access to the vertex value is required, one should use `EdgesFunctionWithVertexValue` and `NeighborsFunctionWithVertexValue` instead.
 
@@ -414,6 +693,8 @@ Let us consider computing Single-Source-Shortest-Paths with vertex-centric itera
     <img alt="Vertex-centric SSSP superstep 2" width="70%" src="fig/gelly-vc-sssp2.png"/>
 </p>
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %} 
 // read the input graph
 Graph<Long, Double, Double> graph = ...
@@ -432,19 +713,19 @@ DataSet<Vertex<Long, Double>> singleSourceShortestPaths = result.getVertices();
 // - - -  UDFs - - - //
 
 // messaging
-public static final class MinDistanceMessenger<K> extends MessagingFunction<K, Double, Double, Double> {
+public static final class MinDistanceMessenger extends MessagingFunction<Long, Double, Double, Double> {
 
-	public void sendMessages(Vertex<K, Double> vertex) {
-		for (Edge<K, Double> edge : getEdges()) {
+	public void sendMessages(Vertex<Long, Double> vertex) {
+		for (Edge<Long, Double> edge : getEdges()) {
 			sendMessageTo(edge.getTarget(), vertex.getValue() + edge.getValue());
 		}
 	}
 }
 
 // vertex update
-public static final class VertexDistanceUpdater<K> extends VertexUpdateFunction<K, Double, Double> {
+public static final class VertexDistanceUpdater extends VertexUpdateFunction<Long, Double, Double> {
 
-	public void updateVertex(Vertex<K, Double> vertex, MessageIterator<Double> inMessages) {
+	public void updateVertex(Vertex<Long, Double> vertex, MessageIterator<Double> inMessages) {
 		Double minDistance = Double.MAX_VALUE;
 
 		for (double msg : inMessages) {
@@ -460,6 +741,56 @@ public static final class VertexDistanceUpdater<K> extends VertexUpdateFunction<
 }
 
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %} 
+// read the input graph
+val graph: Graph[Long, Double, Double] = ...
+
+// define the maximum number of iterations
+val maxIterations = 10
+
+// Execute the vertex-centric iteration
+val result = graph.runVertexCentricIteration(new VertexDistanceUpdater, new MinDistanceMessenger, maxIterations)
+
+// Extract the vertices as the result
+val singleSourceShortestPaths = result.getVertices
+
+
+// - - -  UDFs - - - //
+
+// messaging
+final class MinDistanceMessenger extends MessagingFunction[Long, Double, Double, Double] {
+
+	override def sendMessages(vertex: Vertex[Long, Double]) = {
+		for (edge: Edge[Long, Double] <- getEdges) {
+			sendMessageTo(edge.getTarget, vertex.getValue + edge.getValue)
+		}
+	}
+}
+
+// vertex update
+final class VertexDistanceUpdater extends VertexUpdateFunction[Long, Double, Double] {
+
+	override def updateVertex(vertex: Vertex[Long, Double], inMessages: MessageIterator[Double]) = {
+		var minDistance = Double.MaxValue
+
+		while (inMessages.hasNext) {
+		  val msg = inMessages.next
+		  if (msg < minDistance) {
+			minDistance = msg
+		  }
+		}
+
+		if (vertex.getValue > minDistance) {
+		  setNewVertexValue(minDistance)
+		}
+	}
+}
+{% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
@@ -488,6 +819,8 @@ If the degrees option is not set in the configuration, these methods will return
 
 * <strong>Messaging Direction</strong>: By default, a vertex sends messages to its out-neighbors and updates its value based on messages received from its in-neighbors. This configuration option allows users to change the messaging direction to either `EdgeDirection.IN`, `EdgeDirection.OUT`, `EdgeDirection.ALL`. The messaging direction also dictates the update direction which would be `EdgeDirection.OUT`, `EdgeDirection.IN` and `EdgeDirection.ALL`, respectively. This property can be set using the `setDirection()` method.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 
 Graph<Long, Double, Double> graph = ...
@@ -521,7 +854,7 @@ public static final class VertexUpdater extends VertexUpdateFunction {
 	}
 
 
-	public void updateVertex(Long vertexKey, Long vertexValue, MessageIterator inMessages) {
+	public void updateVertex(Vertex<Long, Long> vertex, MessageIterator inMessages) {
 		
 		//do some computation
 		Long partialValue = ...
@@ -537,9 +870,62 @@ public static final class VertexUpdater extends VertexUpdateFunction {
 public static final class Messenger extends MessagingFunction {...}
 
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+
+val graph: Graph[Long, Double, Double] = ...
+
+val parameters = new VertexCentricConfiguration
+
+// set the iteration name
+parameters.setName("Gelly Iteration")
+
+// set the parallelism
+parameters.setParallelism(16)
+
+// register an aggregator
+parameters.registerAggregator("sumAggregator", new LongSumAggregator)
+
+// run the vertex-centric iteration, also passing the configuration parameters
+val result = graph.runVertexCentricIteration(new VertexUpdater, new Messenger, maxIterations, parameters)
+
+// user-defined functions
+final class VertexUpdater extends VertexUpdateFunction {
+
+	var aggregator = new LongSumAggregator
+
+	override def preSuperstep {
+	
+		// retrieve the Aggregator
+		aggregator = getIterationAggregator("sumAggregator")
+	}
+
+
+	override def updateVertex(vertex: Vertex[Long, Long], inMessages: MessageIterator[Long]) {
+		
+		//do some computation
+		val partialValue = ...
+
+		// aggregate the partial value
+		aggregator.aggregate(partialValue)
+
+		// update the vertex value
+		setNewVertexValue(...)
+	}
+}
+
+final class Messenger extends MessagingFunction {...}
+
+{% endhighlight %}
+</div>
+</div>
 
 The following example illustrates the usage of the degree as well as the number of vertices options.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 
 Graph<Long, Double, Double> graph = ...
@@ -574,11 +960,49 @@ public static final class Messenger {
 }
 
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+
+val graph: Graph[Long, Double, Double] = ...
+
+// configure the iteration
+val parameters = new VertexCentricConfiguration
+
+// set the number of vertices option to true
+parameters.setOptNumVertices(true)
+
+// set the degree option to true
+parameters.setOptDegrees(true)
+
+// run the vertex-centric iteration, also passing the configuration parameters
+val result = graph.runVertexCentricIteration(new VertexUpdater, new Messenger, maxIterations, parameters)
+
+// user-defined functions
+final class VertexUpdater {
+	...
+	// get the number of vertices
+	val numVertices = getNumberOfVertices
+	...
+}
+
+final class Messenger {
+	...
+	// retrieve the vertex out-degree
+	val outDegree = getOutDegree
+	...
+}
+
+{% endhighlight %}
+</div>
+</div>
 
 The following example illustrates the usage of the edge direction option. Vertices update their values to contain a list of all their in-neighbors.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
-
 Graph<Long, HashSet<Long>, Double> graph = ...
 
 // configure the iteration
@@ -594,29 +1018,35 @@ DataSet<Vertex<Long, HashSet<Long>>> result =
 			.getVertices();
 
 // user-defined functions
-public static final class VertexUpdater {
-	@Override
-    public void updateVertex(Vertex<Long, HashSet<Long>> vertex, MessageIterator<Long> messages) throws Exception {
-    	vertex.getValue().clear();
+public static final class VertexUpdater {...}
 
-    	for(long msg : messages) {
-    		vertex.getValue().add(msg);
-    	}
+public static final class Messenger {...}
 
-    	setNewVertexValue(vertex.getValue());
-    }
-}
+{% endhighlight %}
+</div>
 
-public static final class Messenger {
-	@Override
-    public void sendMessages(Vertex<Long, HashSet<Long>> vertex) throws Exception {
-    	for (Edge<Long, Long> edge : getEdges()) {
-    		sendMessageTo(edge.getSource(), vertex.getId());
-    	}
-    }
-}
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val graph: Graph[Long, HashSet[Long], Double] = ...
+
+// configure the iteration
+val parameters = new VertexCentricConfiguration
+
+// set the messaging direction
+parameters.setDirection(EdgeDirection.IN)
+
+// run the vertex-centric iteration, also passing the configuration parameters
+val result = graph.runVertexCentricIteration(new VertexUpdater, new Messenger, maxIterations, parameters)
+			.getVertices
+
+// user-defined functions
+final class VertexUpdater {...}
+
+final class Messenger {...}
 
 {% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
@@ -646,6 +1076,8 @@ The resulting graph after the algorithm converges is shown below.
 
 To implement this example in Gelly GSA, the user only needs to call the `runGatherSumApplyIteration` method on the input graph and provide the `GatherFunction`, `SumFunction` and `ApplyFunction` UDFs. Iteration synchronization, grouping, value updates and convergence are handled by the system:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 // read the input graph
 Graph<Long, Double, Double> graph = ...
@@ -690,10 +1122,58 @@ private static final class UpdateDistance extends ApplyFunction<Long, Double, Do
 }
 
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+// read the input graph
+val graph: Graph[Long, Double, Double] = ...
+
+// define the maximum number of iterations
+val maxIterations = 10
+
+// Execute the GSA iteration
+val result = graph.runGatherSumApplyIteration(new CalculateDistances, new ChooseMinDistance, new UpdateDistance, maxIterations)
+
+// Extract the vertices as the result
+val singleSourceShortestPaths = result.getVertices
+
+
+// - - -  UDFs - - - //
+
+// Gather
+final class CalculateDistances extends GatherFunction[Double, Double, Double] {
+
+	override def gather(neighbor: Neighbor[Double, Double]): Double = {
+		neighbor.getNeighborValue + neighbor.getEdgeValue
+	}
+}
+
+// Sum
+final class ChooseMinDistance extends SumFunction[Double, Double, Double] {
+
+	override def sum(newValue: Double, currentValue: Double): Double = {
+		Math.min(newValue, currentValue)
+	}
+}
+
+// Apply
+final class UpdateDistance extends ApplyFunction[Long, Double, Double] {
+
+	override def apply(newDistance: Double, oldDistance: Double) = {
+		if (newDistance < oldDistance) {
+			setResult(newDistance)
+		}
+	}
+}
+
+{% endhighlight %}
+</div>
+</div>
 
 Note that `gather` takes a `Neighbor` type as an argument. This is a convenience type which simply wraps a vertex with its neighboring edge.
 
-For more examples of how to implement algorithms with the Gather-Sum-Apply model, check the {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSAPageRank.java "GSAPageRank" %} and {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/example/GSAConnectedComponents.java "GSAConnectedComponents" %} examples of Gelly.
+For more examples of how to implement algorithms with the Gather-Sum-Apply model, check the {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAPageRank.java "GSAPageRank" %} and {% gh_link /flink-staging/flink-gelly/src/main/java/org/apache/flink/graph/library/GSAConnectedComponents.java "GSAConnectedComponents" %} library methods of Gelly.
 
 [Back to top](#top)
 
@@ -719,6 +1199,8 @@ using the `setDirection()` method.
 
 The following example illustrates the usage of the number of vertices option.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 
 Graph<Long, Double, Double> graph = ...
@@ -757,8 +1239,51 @@ public static final class Apply {
 }
 
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+
+val graph: Graph[Long, Double, Double] = ...
+
+// configure the iteration
+val parameters = new GSAConfiguration
+
+// set the number of vertices option to true
+parameters.setOptNumVertices(true)
+
+// run the gather-sum-apply iteration, also passing the configuration parameters
+val result = graph.runGatherSumApplyIteration(new Gather, new Sum, new Apply, maxIterations, parameters)
+
+// user-defined functions
+final class Gather {
+	...
+	// get the number of vertices
+	val numVertices = getNumberOfVertices
+	...
+}
+
+final class Sum {
+	...
+    // get the number of vertices
+    val numVertices = getNumberOfVertices
+    ...
+}
+
+final class Apply {
+	...
+    // get the number of vertices
+    val numVertices = getNumberOfVertices
+    ...
+}
+
+{% endhighlight %}
+</div>
+</div>
 
 The following example illustrates the usage of the edge direction option.
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 
 Graph<Long, HashSet<Long>, Double> graph = ...
@@ -775,6 +1300,25 @@ DataSet<Vertex<Long, HashSet<Long>>> result =
 			new Gather(), new Sum(), new Apply(), maxIterations, parameters)
 			.getVertices();
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+
+val graph: Graph[Long, HashSet[Long], Double] = ...
+
+// configure the iteration
+val parameters = new GSAConfiguration
+
+// set the messaging direction
+parameters.setDirection(EdgeDirection.IN)
+
+// run the gather-sum-apply iteration, also passing the configuration parameters
+val result = graph.runGatherSumApplyIteration(new Gather, new Sum, new Apply, maxIterations, parameters)
+			.getVertices()
+{% endhighlight %}
+</div>
+</div>
 [Back to top](#top)
 
 ### Vertex-centric and GSA Comparison
@@ -794,6 +1338,8 @@ Graph Validation
 Gelly provides a simple utility for performing validation checks on input graphs. Depending on the application context, a graph may or may not be valid according to certain criteria. For example, a user might need to validate whether their graph contains duplicate edges or whether its structure is bipartite. In order to validate a graph, one can define a custom `GraphValidator` and implement its `validate()` method. `InvalidVertexIdsValidator` is Gelly's pre-defined validator. It checks that the edge set contains valid vertex IDs, i.e. that all edge IDs
 also exist in the vertex IDs set.
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -809,6 +1355,26 @@ Graph<Long, Long, Long> graph = Graph.fromCollection(vertices, edges, env);
 graph.validate(new InvalidVertexIdsValidator<Long, Long, Long>()); 
 
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+// create a list of vertices with IDs = {1, 2, 3, 4, 5}
+val vertices: List[Vertex[Long, Long]] = ...
+
+// create a list of edges with IDs = {(1, 2) (1, 3), (2, 4), (5, 6)}
+val edges: List[Edge[Long, Long]] = ...
+
+val graph = Graph.fromCollection(vertices, edges, env)
+
+// will return false: 6 is an invalid ID
+graph.validate(new InvalidVertexIdsValidator[Long, Long, Long])
+
+{% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
@@ -828,6 +1394,8 @@ Gelly has a growing collection of graph algorithms for easily analyzing large-sc
 
 Gelly's library methods can be used by simply calling the `run()` method on the input graph:
 
+<div class="codetabs" markdown="1">
+<div data-lang="java" markdown="1">
 {% highlight java %}
 ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 
@@ -842,6 +1410,24 @@ verticesWithCommunity.print();
 
 env.execute();
 {% endhighlight %}
+</div>
+
+<div data-lang="scala" markdown="1">
+{% highlight scala %}
+val env = ExecutionEnvironment.getExecutionEnvironment
+
+val graph: Graph[Long, Long, NullValue] = ...
+
+// run Label Propagation for 30 iterations to detect communities on the input graph
+val verticesWithCommunity = graph.run(new LabelPropagation[Long](30)).getVertices
+
+// print the result
+verticesWithCommunity.print
+
+env.execute
+{% endhighlight %}
+</div>
+</div>
 
 [Back to top](#top)
 
@@ -955,6 +1541,3 @@ The computation **terminates** after a specified *maximum number of supersteps*
 </p>
 
 [Back to top](#top)
-
-
-


[8/9] flink git commit: [FLINK-2778] Add API for non-parallel non-keyed Windows

Posted by se...@apache.org.
[FLINK-2778] Add API for non-parallel non-keyed Windows

This adds two new operators for non-keyed windows: Regular trigger
operator and evicting trigger operator.

This also adds the API calls nonParallelWindow(...) on DataStream and
the API class NonParallelWindowDataStream for representing these
operations.

This also adds tests for both the operators and the translation from API
to operators.


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

Branch: refs/heads/master
Commit: 41717d12372680b9cd55df936c558a255ac25163
Parents: 5623c15
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Sep 29 20:29:09 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 1 11:04:00 2015 +0200

----------------------------------------------------------------------
 .../streaming/api/datastream/DataStream.java    |  77 ++++
 .../datastream/NonParallelWindowDataStream.java | 218 ++++++++++
 .../windowing/ReduceWindowFunction.java         |   4 +-
 .../functions/windowing/RichWindowFunction.java |  25 ++
 .../api/functions/windowing/WindowFunction.java |  43 ++
 .../EvictingNonKeyedWindowOperator.java         | 101 +++++
 .../windowing/NonKeyedWindowOperator.java       | 285 ++++++++++++
 .../EvictingNonKeyedWindowOperatorTest.java     | 173 ++++++++
 .../windowing/NonKeyedWindowOperatorTest.java   | 434 +++++++++++++++++++
 ...ParallelWindowDataStreamTranslationTest.java | 198 +++++++++
 .../windowing/TimeWindowTranslationTest.java    |  45 ++
 .../StreamingScalaAPICompletenessTest.scala     |   6 +-
 12 files changed, 1606 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/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 5dfb1e2..e3f7f3e 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
@@ -60,6 +60,11 @@ import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.transformations.PartitionTransformation;
 import org.apache.flink.streaming.api.transformations.StreamTransformation;
 import org.apache.flink.streaming.api.transformations.UnionTransformation;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+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;
@@ -67,6 +72,10 @@ 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.time.EventTime;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
 import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper;
 import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
@@ -805,6 +814,74 @@ public class DataStream<T> {
 	}
 
 	/**
+	 * Windows this {@code KeyedDataStream} into tumbling time windows.
+	 *
+	 * <p>
+	 * This is a shortcut for either {@code .window(TumblingTimeWindows.of(size))} or
+	 * {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic
+	 * set using
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
+	 *
+	 * @param size The size of the window.
+	 */
+	public NonParallelWindowDataStream<T, TimeWindow> timeWindowAll(AbstractTime size) {
+		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+
+		if (actualSize instanceof EventTime) {
+			return windowAll(TumblingTimeWindows.of(actualSize.toMilliseconds()));
+		} else {
+			return windowAll(TumblingProcessingTimeWindows.of(actualSize.toMilliseconds()));
+		}
+	}
+
+	/**
+	 * Windows this {@code KeyedDataStream} into sliding time windows.
+	 *
+	 * <p>
+	 * This is a shortcut for either {@code .window(SlidingTimeWindows.of(size, slide))} or
+	 * {@code .window(SlidingProcessingTimeWindows.of(size, slide))} depending on the time characteristic
+	 * set using
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
+	 *
+	 * @param size The size of the window.
+	 */
+	public NonParallelWindowDataStream<T, TimeWindow> timeWindowAll(AbstractTime size, AbstractTime slide) {
+		AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+		AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
+
+		if (actualSize instanceof EventTime) {
+			return windowAll(SlidingTimeWindows.of(actualSize.toMilliseconds(),
+					actualSlide.toMilliseconds()));
+		} else {
+			return windowAll(SlidingProcessingTimeWindows.of(actualSize.toMilliseconds(),
+					actualSlide.toMilliseconds()));
+		}
+	}
+
+	/**
+	 * Windows this data stream to a {@code KeyedTriggerWindowDataStream}, which evaluates windows
+	 * over a key grouped stream. Elements are put into windows by a
+	 * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. The grouping of
+	 * elements is done both by key and by window.
+	 *
+	 * <p>
+	 * A {@link org.apache.flink.streaming.api.windowing.triggers.Trigger} can be defined to specify
+	 * when windows are evaluated. However, {@code WindowAssigners} have a default {@code Trigger}
+	 * that is used if a {@code Trigger} is not specified.
+	 *
+	 * <p>
+	 * Note: This operation can be inherently non-parallel since all elements have to pass through
+	 * the same operator instance. (Only for special cases, such as aligned time windows is
+	 * it possible to perform this operation in parallel).
+	 *
+	 * @param assigner The {@code WindowAssigner} that assigns elements to windows.
+	 * @return The trigger windows data stream.
+	 */
+	public <W extends Window> NonParallelWindowDataStream<T, W> windowAll(WindowAssigner<? super T, W> assigner) {
+		return new NonParallelWindowDataStream<>(this, assigner);
+	}
+
+	/**
 	 * Writes a DataStream to the standard output stream (stdout).
 	 *
 	 * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java
new file mode 100644
index 0000000..5cb3b6b
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/NonParallelWindowDataStream.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.commons.lang.SerializationUtils;
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.Utils;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+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.streaming.runtime.operators.windowing.EvictingNonKeyedWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.NonKeyedWindowOperator;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
+
+/**
+ * A {@code NonParallelWindowDataStream} represents a data stream where the stream of
+ * elements is split into windows based on a
+ * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. Window emission
+ * is triggered based on a {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
+ *
+ * <p>
+ * If an {@link org.apache.flink.streaming.api.windowing.evictors.Evictor} is specified it will be
+ * used to evict elements from the window after
+ * evaluation was triggered by the {@code Trigger} but before the actual evaluation of the window.
+ * When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ *
+ * <p>
+ * Note that the {@code NonParallelWindowDataStream} is purely and API construct, during runtime
+ * the {@code NonParallelWindowDataStream} will be collapsed together with the
+ * operation over the window into one single operation.
+ *
+ * @param <T> The type of elements in the stream.
+ * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns the elements to.
+ */
+public class NonParallelWindowDataStream<T, W extends Window> {
+
+	/** The data stream that is windowed by this stream */
+	private final DataStream<T> input;
+
+	/** The window assigner */
+	private final WindowAssigner<? super T, W> windowAssigner;
+
+	/** The trigger that is used for window evaluation/emission. */
+	private Trigger<? super T, ? super W> trigger;
+
+	/** The evictor that is used for evicting elements before window evaluation. */
+	private Evictor<? super T, ? super W> evictor;
+
+
+	public NonParallelWindowDataStream(DataStream<T> input,
+			WindowAssigner<? super T, W> windowAssigner) {
+		this.input = input;
+		this.windowAssigner = windowAssigner;
+		this.trigger = windowAssigner.getDefaultTrigger();
+	}
+
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	public NonParallelWindowDataStream<T, W> trigger(Trigger<? super T, ? super W> trigger) {
+		this.trigger = trigger;
+		return this;
+	}
+
+	/**
+	 * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
+	 *
+	 * <p>
+	 * Note: When using an evictor window performance will degrade significantly, since
+	 * pre-aggregation of window results cannot be used.
+	 */
+	public NonParallelWindowDataStream<T, W> evictor(Evictor<? super T, ? super W> evictor) {
+		this.evictor = evictor;
+		return this;
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  Operations on the keyed windows
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies a reduce function to the window. The window function is called for each evaluation
+	 * of the window for each key individually. The output of the reduce function is interpreted
+	 * as a regular non-windowed stream.
+	 * <p>
+	 * This window will try and pre-aggregate data as much as the window policies permit. For example,
+	 * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
+	 * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide interval,
+	 * so a few elements are stored per key (one per slide interval).
+	 * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
+	 * aggregation tree.
+	 * 
+	 * @param function The reduce function.
+	 * @return The data stream that is the result of applying the reduce function to the window. 
+	 */
+	public DataStream<T> reduceWindow(ReduceFunction<T> function) {
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "Reduce at " + callLocation;
+
+		DataStream<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
+		if (result != null) {
+			return result;
+		}
+
+		String opName = "NonParallelTriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+
+		OneInputStreamOperator<T, T> operator;
+
+		if (evictor != null) {
+			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
+					new HeapWindowBuffer.Factory<T>(),
+					new ReduceWindowFunction<W, T>(function),
+					trigger,
+					evictor);
+
+		} else {
+			// we need to copy because we need our own instance of the pre aggregator
+			@SuppressWarnings("unchecked")
+			ReduceFunction<T> functionCopy = (ReduceFunction<T>) SerializationUtils.clone(function);
+
+			operator = new NonKeyedWindowOperator<>(windowAssigner,
+					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
+					new ReduceWindowFunction<W, T>(function),
+					trigger);
+		}
+
+		return input.transform(opName, input.getType(), operator).setParallelism(1);
+	}
+
+	/**
+	 * Applies a window function to the 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.
+	 * <p>
+	 * 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.
+	 */
+	public <R> DataStream<R> mapWindow(WindowFunction<T, R, W> function) {
+		TypeInformation<T> inType = input.getType();
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
+				function, WindowFunction.class, true, true, inType, null, false);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "MapWindow at " + callLocation;
+
+		DataStream<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		if (result != null) {
+			return result;
+		}
+
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+
+		OneInputStreamOperator<T, R> operator;
+
+		if (evictor != null) {
+			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor);
+
+		} else {
+			operator = new NonKeyedWindowOperator<>(windowAssigner,
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger);
+		}
+
+
+
+		return input.transform(opName, resultType, operator).setParallelism(1);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+
+	private <R> DataStream<R> createFastTimeOperatorIfValid(
+			Function function,
+			TypeInformation<R> resultType,
+			String functionName) {
+
+		// TODO: add once non-parallel fast aligned time windows operator is ready
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
index 1c9578a..ba26218 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
@@ -24,7 +24,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
-public class ReduceWindowFunction<K, W extends Window, T> extends RichKeyedWindowFunction<T, T, K, W> {
+public class ReduceWindowFunction<W extends Window, T> extends RichWindowFunction<T, T, W> {
 	private static final long serialVersionUID = 1L;
 
 	private final ReduceFunction<T> reduceFunction;
@@ -52,7 +52,7 @@ public class ReduceWindowFunction<K, W extends Window, T> extends RichKeyedWindo
 	}
 
 	@Override
-	public void evaluate(K k, W window, Iterable<T> values, Collector<T> out) throws Exception {
+	public void evaluate(W window, Iterable<T> values, Collector<T> out) throws Exception {
 		T result = null;
 
 		for (T v: values) {

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java
new file mode 100644
index 0000000..b40b74a
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+
+public abstract class RichWindowFunction<IN, OUT, W extends Window> extends AbstractRichFunction implements WindowFunction<IN, OUT, W> {
+	private static final long serialVersionUID = 1L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java
new file mode 100644
index 0000000..1a4304e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.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;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.io.Serializable;
+
+/**
+ * Base interface for functions that are evaluated over non-keyed windows.
+ *
+ * @param <IN> The type of the input value.
+ * @param <OUT> The type of the output value.
+ */
+public interface WindowFunction<IN, OUT,  W extends Window> extends Function, Serializable {
+
+	/**
+	 * 
+	 * @param values
+	 * @param out
+	 * 
+	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery. 
+	 */
+	void evaluate(W window, Iterable<IN> values, Collector<OUT> out) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
new file mode 100644
index 0000000..d5ed6cb
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
@@ -0,0 +1,101 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.flink.streaming.runtime.operators.windowing;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+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.streaming.runtime.operators.windowing.buffers.EvictingWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class EvictingNonKeyedWindowOperator<IN, OUT, W extends Window> extends NonKeyedWindowOperator<IN, OUT, W> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(EvictingNonKeyedWindowOperator.class);
+
+	private final Evictor<? super IN, ? super W> evictor;
+
+	public EvictingNonKeyedWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
+			WindowBufferFactory<? super IN, ? extends EvictingWindowBuffer<IN>> windowBufferFactory,
+			WindowFunction<IN, OUT, W> windowFunction,
+			Trigger<? super IN, ? super W> trigger,
+			Evictor<? super IN, ? super W> evictor) {
+		super(windowAssigner, windowBufferFactory, windowFunction, trigger);
+		this.evictor = evictor;
+	}
+
+	@Override
+	@SuppressWarnings("unchecked, rawtypes")
+	protected void emitWindow(W window, boolean purge) throws Exception {
+
+		timestampedCollector.setTimestamp(window.getEnd());
+
+		Tuple2<WindowBuffer<IN>, TriggerContext> bufferAndTrigger;
+		if (purge) {
+			bufferAndTrigger = windows.remove(window);
+		} else {
+			bufferAndTrigger = windows.get(window);
+		}
+
+		if (bufferAndTrigger == null) {
+			LOG.debug("Window {} already gone.", window);
+			return;
+		}
+
+
+		EvictingWindowBuffer<IN> windowBuffer = (EvictingWindowBuffer<IN>) bufferAndTrigger.f0;
+
+		int toEvict = 0;
+		if (windowBuffer.size() > 0) {
+			// need some type trickery here...
+			toEvict = evictor.evict((Iterable) windowBuffer.getElements(), windowBuffer.size(), window);
+		}
+
+		windowBuffer.removeElements(toEvict);
+
+		userFunction.evaluate(
+				window,
+				bufferAndTrigger.f0.getUnpackedElements(),
+				timestampedCollector);
+	}
+
+	@Override
+	public EvictingNonKeyedWindowOperator<IN, OUT, W> enableSetProcessingTime(boolean setProcessingTime) {
+		super.enableSetProcessingTime(setProcessingTime);
+		return this;
+	}
+
+
+	// ------------------------------------------------------------------------
+	// Getters for testing
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	public Evictor<? super IN, ? super W> getEvictor() {
+		return evictor;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
new file mode 100644
index 0000000..3a85759
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
@@ -0,0 +1,285 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS 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.runtime.operators.windowing;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+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.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+public class NonKeyedWindowOperator<IN, OUT, W extends Window>
+		extends AbstractUdfStreamOperator<OUT, WindowFunction<IN, OUT, W>>
+		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(NonKeyedWindowOperator.class);
+
+
+	private final WindowAssigner<? super IN, W> windowAssigner;
+
+	private final Trigger<? super IN, ? super W> triggerTemplate;
+	private final WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory;
+
+	protected transient Map<W, Tuple2<WindowBuffer<IN>, TriggerContext>> windows;
+
+	private transient Map<Long, Set<TriggerContext>> processingTimeTimers;
+	private transient Map<Long, Set<TriggerContext>> watermarkTimers;
+
+	protected transient TimestampedCollector<OUT> timestampedCollector;
+
+	private boolean setProcessingTime = false;
+
+	private TypeSerializer<IN> inputSerializer;
+
+	public NonKeyedWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
+			WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory,
+			WindowFunction<IN, OUT, W> windowFunction,
+			Trigger<? super IN, ? super W> trigger) {
+
+		super(windowFunction);
+
+		this.windowAssigner = windowAssigner;
+
+		this.windowBufferFactory = windowBufferFactory;
+		this.triggerTemplate = trigger;
+
+		setChainingStrategy(ChainingStrategy.ALWAYS);
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
+		inputSerializer = (TypeSerializer<IN>) type.createSerializer(executionConfig);
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		windows = Maps.newHashMap();
+		watermarkTimers = Maps.newHashMap();
+		processingTimeTimers = Maps.newHashMap();
+		timestampedCollector = new TimestampedCollector<>(output);
+
+		if (inputSerializer == null) {
+			throw new IllegalStateException("Input serializer was not set.");
+		}
+
+		windowBufferFactory.setRuntimeContext(getRuntimeContext());
+		windowBufferFactory.open(parameters);
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		// emit the elements that we still keep
+		for (W window: windows.keySet()) {
+			emitWindow(window, false);
+		}
+		windows.clear();
+		windowBufferFactory.close();
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public void processElement(StreamRecord<IN> element) throws Exception {
+		if (setProcessingTime) {
+			element.replace(element.getValue(), System.currentTimeMillis());
+		}
+		Collection<W> elementWindows = windowAssigner.assignWindows(element.getValue(), element.getTimestamp());
+
+		for (W window: elementWindows) {
+			Tuple2<WindowBuffer<IN>, TriggerContext> bufferAndTrigger = windows.get(window);
+			if (bufferAndTrigger == null) {
+				bufferAndTrigger = new Tuple2<>();
+				bufferAndTrigger.f0 = windowBufferFactory.create();
+				bufferAndTrigger.f1 = new TriggerContext(window, triggerTemplate.duplicate());
+				windows.put(window, bufferAndTrigger);
+			}
+			StreamRecord<IN> elementCopy = new StreamRecord<>(inputSerializer.copy(element.getValue()), element.getTimestamp());
+			bufferAndTrigger.f0.storeElement(elementCopy);
+			Trigger.TriggerResult triggerResult = bufferAndTrigger.f1.trigger.onElement(elementCopy.getValue(), elementCopy.getTimestamp(), window, bufferAndTrigger.f1);
+			processTriggerResult(triggerResult, window);
+		}
+	}
+
+	protected void emitWindow(W window, boolean purge) throws Exception {
+		timestampedCollector.setTimestamp(window.getEnd());
+
+		Tuple2<WindowBuffer<IN>, TriggerContext> bufferAndTrigger;
+		if (purge) {
+			bufferAndTrigger = windows.remove(window);
+		} else {
+			bufferAndTrigger = windows.get(window);
+		}
+
+		if (bufferAndTrigger == null) {
+			LOG.debug("Window {} already gone.", window);
+			return;
+		}
+
+
+		userFunction.evaluate(
+				window,
+				bufferAndTrigger.f0.getUnpackedElements(),
+				timestampedCollector);
+	}
+
+	private void processTriggerResult(Trigger.TriggerResult triggerResult, W window) throws Exception {
+		switch (triggerResult) {
+			case FIRE:
+				emitWindow(window, false);
+				break;
+
+			case FIRE_AND_PURGE:
+				emitWindow(window, true);
+				break;
+
+			case CONTINUE:
+				// ingore
+		}
+	}
+
+	@Override
+	public void processWatermark(Watermark mark) throws Exception {
+		Set<Long> toRemove = Sets.newHashSet();
+
+		for (Map.Entry<Long, Set<TriggerContext>> triggers: watermarkTimers.entrySet()) {
+			if (triggers.getKey() <= mark.getTimestamp()) {
+				for (TriggerContext trigger: triggers.getValue()) {
+					Trigger.TriggerResult triggerResult = trigger.trigger.onTime(mark.getTimestamp(), trigger);
+					processTriggerResult(triggerResult, trigger.window);
+				}
+				toRemove.add(triggers.getKey());
+			}
+		}
+
+		for (Long l: toRemove) {
+			watermarkTimers.remove(l);
+		}
+		output.emitWatermark(mark);
+	}
+
+	@Override
+	public void trigger(long time) throws Exception {
+		Set<Long> toRemove = Sets.newHashSet();
+
+		for (Map.Entry<Long, Set<TriggerContext>> triggers: processingTimeTimers.entrySet()) {
+			if (triggers.getKey() < time) {
+				for (TriggerContext trigger: triggers.getValue()) {
+					Trigger.TriggerResult triggerResult = trigger.trigger.onTime(time, trigger);
+					processTriggerResult(triggerResult, trigger.window);
+				}
+				toRemove.add(triggers.getKey());
+			}
+		}
+
+		for (Long l: toRemove) {
+			processingTimeTimers.remove(l);
+		}
+	}
+
+	protected class TriggerContext implements Trigger.TriggerContext {
+		Trigger<? super IN, ? super W> trigger;
+		W window;
+
+		public TriggerContext(W window, Trigger<? super IN, ? super W> trigger) {
+			this.window = window;
+			this.trigger = trigger;
+		}
+
+		@Override
+		public void registerProcessingTimeTimer(long time) {
+			Set<TriggerContext> triggers = processingTimeTimers.get(time);
+			if (triggers == null) {
+				getRuntimeContext().registerTimer(time, NonKeyedWindowOperator.this);
+				triggers = Sets.newHashSet();
+				processingTimeTimers.put(time, triggers);
+			}
+			triggers.add(this);
+		}
+
+		@Override
+		public void registerWatermarkTimer(long time) {
+			Set<TriggerContext> triggers = watermarkTimers.get(time);
+			if (triggers == null) {
+				triggers = Sets.newHashSet();
+				watermarkTimers.put(time, triggers);
+			}
+			triggers.add(this);
+		}
+	}
+
+	/**
+	 * When this flag is enabled the current processing time is set as the timestamp of elements
+	 * upon arrival. This must be used, for example, when using the
+	 * {@link org.apache.flink.streaming.api.windowing.evictors.TimeEvictor} with processing
+	 * time semantics.
+	 */
+	public NonKeyedWindowOperator<IN, OUT, W> enableSetProcessingTime(boolean setProcessingTime) {
+		this.setProcessingTime = setProcessingTime;
+		return this;
+	}
+
+	// ------------------------------------------------------------------------
+	// Getters for testing
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	public Trigger<? super IN, ? super W> getTriggerTemplate() {
+		return triggerTemplate;
+	}
+
+	@VisibleForTesting
+	public WindowAssigner<? super IN, W> getWindowAssigner() {
+		return windowAssigner;
+	}
+
+	@VisibleForTesting
+	public WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> getWindowBufferFactory() {
+		return windowBufferFactory;
+	}
+
+	@VisibleForTesting
+	public boolean isSetProcessingTime() {
+		return setProcessingTime;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
new file mode 100644
index 0000000..0dfceab
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.runtime.operators.windowing;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+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.api.windowing.windows.GlobalWindow;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class EvictingNonKeyedWindowOperatorTest {
+
+	// For counting if close() is called the correct number of times on the SumReducer
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testCountTrigger() throws Exception {
+		AtomicInteger closeCalled = new AtomicInteger(0);
+
+		final int WINDOW_SIZE = 4;
+		final int WINDOW_SLIDE = 2;
+
+		EvictingNonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new EvictingNonKeyedWindowOperator<>(
+				GlobalWindows.create(),
+				new HeapWindowBuffer.Factory<Tuple2<String, Integer>>(),
+				new ReduceWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer(closeCalled)),
+				CountTrigger.of(WINDOW_SLIDE),
+				CountEvictor.of(WINDOW_SIZE));
+
+		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
+
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new OneInputStreamOperatorTestHarness<>(operator);
+
+		long initialTime = 0L;
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		// The global window actually ignores these timestamps...
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3999));
+
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+
+
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), Long.MAX_VALUE));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), Long.MAX_VALUE));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), Long.MAX_VALUE));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.close();
+
+		Assert.assertEquals("Close was not called.", 1, closeCalled.get());
+
+
+	}
+
+	// ------------------------------------------------------------------------
+	//  UDFs
+	// ------------------------------------------------------------------------
+
+	public static class SumReducer extends RichReduceFunction<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		private boolean openCalled = false;
+
+		private  AtomicInteger closeCalled;
+
+		public SumReducer(AtomicInteger closeCalled) {
+			this.closeCalled = closeCalled;
+		}
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+			openCalled = true;
+		}
+
+		@Override
+		public void close() throws Exception {
+			super.close();
+			closeCalled.incrementAndGet();
+		}
+
+		@Override
+		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
+				Tuple2<String, Integer> value2) throws Exception {
+			if (!openCalled) {
+				Assert.fail("Open was not called");
+			}
+			return new Tuple2<>(value2.f0, value1.f1 + value2.f1);
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	private static class ResultSortComparator implements Comparator<Object> {
+		@Override
+		public int compare(Object o1, Object o2) {
+			if (o1 instanceof Watermark || o2 instanceof Watermark) {
+				return 0;
+			} else {
+				StreamRecord<Tuple2<String, Integer>> sr0 = (StreamRecord<Tuple2<String, Integer>>) o1;
+				StreamRecord<Tuple2<String, Integer>> sr1 = (StreamRecord<Tuple2<String, Integer>>) o2;
+				if (sr0.getTimestamp() != sr1.getTimestamp()) {
+					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
+				}
+				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
+				if (comparison != 0) {
+					return comparison;
+				} else {
+					return sr0.getValue().f1 - sr1.getValue().f1;
+				}
+			}
+		}
+	}
+
+	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(Tuple2<String, Integer> value) throws Exception {
+			return value.f0;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
new file mode 100644
index 0000000..b74b3ea
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
@@ -0,0 +1,434 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.runtime.operators.windowing;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+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.triggers.ContinuousWatermarkTrigger;
+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.triggers.WatermarkTrigger;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@RunWith(Parameterized.class)
+public class NonKeyedWindowOperatorTest {
+
+	@SuppressWarnings("unchecked,rawtypes")
+	private WindowBufferFactory windowBufferFactory;
+
+	public NonKeyedWindowOperatorTest(WindowBufferFactory<?, ?> windowBufferFactory) {
+		this.windowBufferFactory = windowBufferFactory;
+	}
+
+	// For counting if close() is called the correct number of times on the SumReducer
+	private static AtomicInteger closeCalled = new AtomicInteger(0);
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testSlidingEventTimeWindows() throws Exception {
+		closeCalled.set(0);
+
+		final int WINDOW_SIZE = 3000;
+		final int WINDOW_SLIDE = 1000;
+
+		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
+				SlidingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE),
+				windowBufferFactory,
+				new ReduceWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				WatermarkTrigger.create());
+
+		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new OneInputStreamOperatorTestHarness<>(operator);
+
+		long initialTime = 0L;
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3000));
+
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+
+
+		testHarness.processWatermark(new Watermark(initialTime + 999));
+		expectedOutput.add(new Watermark(999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+
+		testHarness.processWatermark(new Watermark(initialTime + 1999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), initialTime + 1999));
+		expectedOutput.add(new Watermark(1999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), initialTime + 2999));
+		expectedOutput.add(new Watermark(2999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 3999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 5), initialTime + 3999));
+		expectedOutput.add(new Watermark(3999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 4999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), initialTime + 4999));
+		expectedOutput.add(new Watermark(4999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), initialTime + 5999));
+		expectedOutput.add(new Watermark(5999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+
+		// those don't have any effect...
+		testHarness.processWatermark(new Watermark(initialTime + 6999));
+		testHarness.processWatermark(new Watermark(initialTime + 7999));
+		expectedOutput.add(new Watermark(6999));
+		expectedOutput.add(new Watermark(7999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.close();
+		if (windowBufferFactory instanceof PreAggregatingHeapWindowBuffer.Factory) {
+			Assert.assertEquals("Close was not called.", 2, closeCalled.get());
+		} else {
+			Assert.assertEquals("Close was not called.", 1, closeCalled.get());
+		}
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testTumblingEventTimeWindows() throws Exception {
+		closeCalled.set(0);
+
+		final int WINDOW_SIZE = 3000;
+
+		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
+				TumblingTimeWindows.of(WINDOW_SIZE),
+				windowBufferFactory,
+				new ReduceWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				WatermarkTrigger.create());
+
+		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
+
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new OneInputStreamOperatorTestHarness<>(operator);
+
+		long initialTime = 0L;
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3000));
+
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+
+
+		testHarness.processWatermark(new Watermark(initialTime + 999));
+		expectedOutput.add(new Watermark(999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+
+		testHarness.processWatermark(new Watermark(initialTime + 1999));
+		expectedOutput.add(new Watermark(1999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 2999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 3), initialTime + 2999));
+		expectedOutput.add(new Watermark(2999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 3999));
+		expectedOutput.add(new Watermark(3999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 4999));
+		expectedOutput.add(new Watermark(4999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 5999));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 2), initialTime + 5999));
+		expectedOutput.add(new Watermark(5999));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+
+		// those don't have any effect...
+		testHarness.processWatermark(new Watermark(initialTime + 6999));
+		testHarness.processWatermark(new Watermark(initialTime + 7999));
+		expectedOutput.add(new Watermark(6999));
+		expectedOutput.add(new Watermark(7999));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.close();
+		if (windowBufferFactory instanceof PreAggregatingHeapWindowBuffer.Factory) {
+			Assert.assertEquals("Close was not called.", 2, closeCalled.get());
+		} else {
+			Assert.assertEquals("Close was not called.", 1, closeCalled.get());
+		}
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testContinuousWatermarkTrigger() throws Exception {
+		closeCalled.set(0);
+
+		final int WINDOW_SIZE = 3000;
+
+		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new NonKeyedWindowOperator<>(
+				GlobalWindows.create(),
+				windowBufferFactory,
+				new ReduceWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				ContinuousWatermarkTrigger.of(WINDOW_SIZE));
+
+		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new OneInputStreamOperatorTestHarness<>(operator);
+
+		long initialTime = 0L;
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		// The global window actually ignores these timestamps...
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3999));
+
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+
+
+		testHarness.processWatermark(new Watermark(initialTime + 1000));
+		expectedOutput.add(new Watermark(1000));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+
+		testHarness.processWatermark(new Watermark(initialTime + 2000));
+		expectedOutput.add(new Watermark(2000));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 3000));
+		expectedOutput.add(new Watermark(3000));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 4000));
+		expectedOutput.add(new Watermark(4000));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 5000));
+		expectedOutput.add(new Watermark(5000));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.processWatermark(new Watermark(initialTime + 6000));
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 5), Long.MAX_VALUE));
+		expectedOutput.add(new Watermark(6000));
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+
+		// those don't have any effect...
+		testHarness.processWatermark(new Watermark(initialTime + 7000));
+		testHarness.processWatermark(new Watermark(initialTime + 8000));
+		expectedOutput.add(new Watermark(7000));
+		expectedOutput.add(new Watermark(8000));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.close();
+		if (windowBufferFactory instanceof PreAggregatingHeapWindowBuffer.Factory) {
+			Assert.assertEquals("Close was not called.", 2, closeCalled.get());
+		} else {
+			Assert.assertEquals("Close was not called.", 1, closeCalled.get());
+		}
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testCountTrigger() throws Exception {
+		closeCalled.set(0);
+
+		final int WINDOW_SIZE = 4;
+
+		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new NonKeyedWindowOperator<>(
+				GlobalWindows.create(),
+				windowBufferFactory,
+				new ReduceWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				PurgingTrigger.of(CountTrigger.of(WINDOW_SIZE)));
+
+		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(
+				"Tuple2<String, Integer>"), new ExecutionConfig());
+
+		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
+				new OneInputStreamOperatorTestHarness<>(operator);
+
+		long initialTime = 0L;
+		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+		testHarness.open();
+
+		// The global window actually ignores these timestamps...
+
+		// add elements out-of-order
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 3999));
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1998));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1999));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), Long.MAX_VALUE));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
+
+		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), Long.MAX_VALUE));
+
+		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+		testHarness.close();
+		if (windowBufferFactory instanceof PreAggregatingHeapWindowBuffer.Factory) {
+			Assert.assertEquals("Close was not called.", 2, closeCalled.get());
+		} else {
+			Assert.assertEquals("Close was not called.", 1, closeCalled.get());
+		}
+
+	}
+
+	// ------------------------------------------------------------------------
+	//  UDFs
+	// ------------------------------------------------------------------------
+
+	public static class SumReducer extends RichReduceFunction<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		private boolean openCalled = false;
+
+		@Override
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+			openCalled = true;
+		}
+
+		@Override
+		public void close() throws Exception {
+			super.close();
+			closeCalled.incrementAndGet();
+		}
+
+		@Override
+		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1,
+				Tuple2<String, Integer> value2) throws Exception {
+			if (!openCalled) {
+				Assert.fail("Open was not called");
+			}
+			return new Tuple2<>(value2.f0, value1.f1 + value2.f1);
+		}
+	}
+	// ------------------------------------------------------------------------
+	//  Parametrization for testing different window buffers
+	// ------------------------------------------------------------------------
+
+	@Parameterized.Parameters(name = "WindowBuffer = {0}")
+	@SuppressWarnings("unchecked,rawtypes")
+	public static Collection<WindowBufferFactory[]> windowBuffers(){
+		return Arrays.asList(new WindowBufferFactory[]{new PreAggregatingHeapWindowBuffer.Factory(new SumReducer())},
+				new WindowBufferFactory[]{new HeapWindowBuffer.Factory()}
+				);
+	}
+
+	@SuppressWarnings("unchecked")
+	private static class ResultSortComparator implements Comparator<Object> {
+		@Override
+		public int compare(Object o1, Object o2) {
+			if (o1 instanceof Watermark || o2 instanceof Watermark) {
+				return 0;
+			} else {
+				StreamRecord<Tuple2<String, Integer>> sr0 = (StreamRecord<Tuple2<String, Integer>>) o1;
+				StreamRecord<Tuple2<String, Integer>> sr1 = (StreamRecord<Tuple2<String, Integer>>) o2;
+				if (sr0.getTimestamp() != sr1.getTimestamp()) {
+					return (int) (sr0.getTimestamp() - sr1.getTimestamp());
+				}
+				int comparison = sr0.getValue().f0.compareTo(sr1.getValue().f0);
+				if (comparison != 0) {
+					return comparison;
+				} else {
+					return sr0.getValue().f1 - sr1.getValue().f1;
+				}
+			}
+		}
+	}
+
+	private static class TupleKeySelector implements KeySelector<Tuple2<String, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(Tuple2<String, Integer> value) throws Exception {
+			return value.f0;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java
new file mode 100644
index 0000000..4babee1
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonParallelWindowDataStreamTranslationTest.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.runtime.operators.windowing;
+
+import org.apache.flink.api.common.functions.RichReduceFunction;
+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.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
+import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * These tests verify that the api calls on
+ * {@link org.apache.flink.streaming.api.datastream.KeyedWindowDataStream} instantiate
+ * the correct window operator.
+ */
+public class NonParallelWindowDataStreamTranslationTest extends StreamingMultipleProgramsTestBase {
+
+	/**
+	 * These tests ensure that the fast aligned time windows operator is used if the
+	 * conditions are right.
+	 *
+	 * TODO: update once fast aligned time windows operator is in
+	 */
+	@Ignore
+	@Test
+	public void testFastTimeWindows() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
+	}
+
+	@Test
+	@SuppressWarnings("rawtypes")
+	public void testNonEvicting() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.trigger(CountTrigger.of(100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof NonKeyedWindowOperator);
+		NonKeyedWindowOperator winOperator1 = (NonKeyedWindowOperator) operator1;
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.windowAll(TumblingProcessingTimeWindows.of(1000))
+				.trigger(CountTrigger.of(100))
+				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof NonKeyedWindowOperator);
+		NonKeyedWindowOperator winOperator2 = (NonKeyedWindowOperator) operator2;
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+	}
+
+	@Test
+	@SuppressWarnings("rawtypes")
+	public void testEvicting() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.windowAll(SlidingProcessingTimeWindows.of(1000, 100))
+				.evictor(CountEvictor.of(100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof EvictingNonKeyedWindowOperator);
+		EvictingNonKeyedWindowOperator winOperator1 = (EvictingNonKeyedWindowOperator) operator1;
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.windowAll(TumblingProcessingTimeWindows.of(1000))
+				.trigger(CountTrigger.of(100))
+				.evictor(TimeEvictor.of(100))
+				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof EvictingNonKeyedWindowOperator);
+		EvictingNonKeyedWindowOperator winOperator2 = (EvictingNonKeyedWindowOperator) operator2;
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+	}
+
+	// ------------------------------------------------------------------------
+	//  UDFs
+	// ------------------------------------------------------------------------
+
+	public static class DummyReducer extends RichReduceFunction<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
+			return value1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
index d0aa547..aaf21e0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
@@ -23,6 +23,7 @@ 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.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.windowing.time.Time;
@@ -30,6 +31,7 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.concurrent.TimeUnit;
@@ -82,6 +84,49 @@ public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase
 		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
 	}
 
+	/**
+	 * These tests ensure that the fast aligned time windows operator is used if the
+	 * conditions are right.
+	 *
+	 * TODO: update once the fast aligned time windows operator is in
+	 */
+	@Ignore
+	@Test
+	public void testNonParallelFastTimeWindows() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS),
+						Time.of(100, TimeUnit.MILLISECONDS))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS))
+				.mapWindow(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
+	}
+
 	// ------------------------------------------------------------------------
 	//  UDFs
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/41717d12/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 e7d9953..f53b986 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
@@ -62,7 +62,11 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
 
 
       // TypeHints are only needed for Java API, Scala API doesn't need them
-      "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.returns"
+      "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.returns",
+
+      // Deactivated until Scala API has new windowing API
+      "org.apache.flink.streaming.api.datastream.DataStream.timeWindowAll",
+      "org.apache.flink.streaming.api.datastream.DataStream.windowAll"
     )
     val excludedPatterns = Seq(
       // We don't have project on tuples in the Scala API


[2/9] flink git commit: [streaming] [storm] Clean up instantiation of mini clusters and test environments.

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/client/RemoteTezEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/client/RemoteTezEnvironment.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/client/RemoteTezEnvironment.java
index a02b536..131937e 100644
--- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/client/RemoteTezEnvironment.java
+++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/client/RemoteTezEnvironment.java
@@ -73,7 +73,7 @@ public class RemoteTezEnvironment extends ExecutionEnvironment {
 
 	public RemoteTezEnvironment() {
 		compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), new org.apache.flink.configuration.Configuration());
-		executor = new TezExecutor(compiler, this.getDegreeOfParallelism());
+		executor = new TezExecutor(compiler, getParallelism());
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java b/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
index 70c5492..e3c6f1b 100644
--- a/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
+++ b/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
@@ -20,6 +20,7 @@ package org.apache.flink.tez.test;
 
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.tez.client.LocalTezEnvironment;
 import org.junit.Assert;
@@ -40,7 +41,7 @@ public abstract class TezProgramTestBase extends AbstractTestBase {
     }
 
     public TezProgramTestBase(Configuration config) {
-        super (config);
+        super (config, StreamingMode.BATCH_ONLY);
     }
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
index 1e85c71..005382a 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
@@ -28,28 +28,36 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 import org.apache.flink.runtime.akka.AkkaUtils;
 
-
+/**
+ * A base class for tests that run test programs in a Flink mini cluster.
+ */
 public abstract class AbstractTestBase extends TestBaseUtils {
-
-
+	
+	/** Configuration to start the testing cluster with */
 	protected final Configuration config;
-
+	
 	private final List<File> tempFiles;
-
-	private final FiniteDuration timeout;
 	
-	protected int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
+	private final FiniteDuration timeout;
 
-	protected int numTaskManagers = DEFAULT_NUM_TASK_MANAGERS;
+	/** Mode (batch-only / streaming) in which to start the system */
+	private final StreamingMode streamingMode;
+
+	protected int taskManagerNumSlots = 1;
+
+	protected int numTaskManagers = 1;
 	
+	/** The mini cluster that runs the test programs */
 	protected ForkableFlinkMiniCluster executor;
 	
 
-	public AbstractTestBase(Configuration config) {
-		this.config = config;
+	public AbstractTestBase(Configuration config, StreamingMode streamingMode) {
+		this.config = Objects.requireNonNull(config);
+		this.streamingMode = Objects.requireNonNull(streamingMode);
 		this.tempFiles = new ArrayList<File>();
 
 		timeout = AkkaUtils.getTimeout(config);
@@ -59,11 +67,11 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	//  Local Test Cluster Life Cycle
 	// --------------------------------------------------------------------------------------------
 
-	public void startCluster() throws Exception{
+	public void startCluster() throws Exception {
 		this.executor = startCluster(
 			numTaskManagers,
 			taskManagerNumSlots,
-			StreamingMode.BATCH_ONLY,
+			streamingMode,
 			false,
 			false,
 			true);
@@ -71,7 +79,6 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 
 	public void stopCluster() throws Exception {
 		stopCluster(executor, timeout);
-
 		deleteAllTempFiles();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
index e639c80..f2de650 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/JavaProgramTestBase.java
@@ -22,6 +22,7 @@ import java.util.Comparator;
 
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
 import org.junit.Assert;
 import org.junit.Test;
 import org.apache.flink.api.java.tuple.Tuple;
@@ -50,7 +51,7 @@ public abstract class JavaProgramTestBase extends AbstractTestBase {
 	}
 	
 	public JavaProgramTestBase(Configuration config) {
-		super(config);
+		super(config, StreamingMode.BATCH_ONLY);
 		setTaskManagerNumSlots(parallelism);
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
index bd5400d..70eeffd 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/RecordAPITestBase.java
@@ -27,6 +27,7 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 
 import org.junit.Assert;
@@ -46,7 +47,7 @@ public abstract class RecordAPITestBase extends AbstractTestBase {
 	}
 	
 	public RecordAPITestBase(Configuration config) {
-		super(config);
+		super(config, StreamingMode.BATCH_ONLY);
 		setTaskManagerNumSlots(parallelism);
 	}
 	
@@ -67,10 +68,11 @@ public abstract class RecordAPITestBase extends AbstractTestBase {
 	
 	protected JobGraph getJobGraph() throws Exception {
 		Plan p = getTestJob();
-		p.setExecutionConfig(new ExecutionConfig());
 		if (p == null) {
-			Assert.fail("Error: Cannot obtain Pact plan. Did the test forget to override either 'getPactPlan()' or 'getJobGraph()' ?");
+			Assert.fail("Error: Cannot obtain plan. Did the test forget to override either 'getPactPlan()' or 'getJobGraph()' ?");
 		}
+
+		p.setExecutionConfig(new ExecutionConfig());
 		
 		Optimizer pc = new Optimizer(new DataStatistics(), this.config);
 		OptimizedPlan op = pc.compile(p);

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index a2c7b93..49cc68b 100644
--- a/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -78,10 +78,6 @@ public class TestBaseUtils extends TestLogger {
 
 	protected static final long TASK_MANAGER_MEMORY_SIZE = 80;
 
-	protected static final int DEFAULT_TASK_MANAGER_NUM_SLOTS = 1;
-
-	protected static final int DEFAULT_NUM_TASK_MANAGERS = 1;
-
 	protected static final long DEFAULT_AKKA_ASK_TIMEOUT = 1000;
 
 	protected static final String DEFAULT_AKKA_STARTUP_TIMEOUT = "60 s";
@@ -118,7 +114,7 @@ public class TestBaseUtils extends TestLogger {
 		
 		config.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, startWebserver);
 
-		if(startZooKeeper) {
+		if (startZooKeeper) {
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
 			config.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
 		}
@@ -159,7 +155,8 @@ public class TestBaseUtils extends TestLogger {
 		if (executor != null) {
 			int numUnreleasedBCVars = 0;
 			int numActiveConnections = 0;
-			{
+			
+			if (executor.running()) {
 				List<ActorRef> tms = executor.getTaskManagersAsJava();
 				List<Future<Object>> bcVariableManagerResponseFutures = new ArrayList<Future<Object>>();
 				List<Future<Object>> numActiveConnectionsResponseFutures = new ArrayList<Future<Object>>();

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java
index f4b3875..a185135 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/ExecutionEnvironmentITCase.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.test.javaApiOperators;
 
 import org.apache.flink.api.common.ExecutionConfig;
@@ -22,44 +23,26 @@ import org.apache.flink.api.common.functions.RichMapPartitionFunction;
 import org.apache.flink.api.common.io.GenericInputFormat;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
 import org.apache.flink.util.Collector;
+
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 
-
 /**
  * Test ExecutionEnvironment from user perspective
  */
 @SuppressWarnings("serial")
-@RunWith(Parameterized.class)
-public class ExecutionEnvironmentITCase extends MultipleProgramsTestBase {
+public class ExecutionEnvironmentITCase {
+	
 	private static final int PARALLELISM = 5;
 
-	public ExecutionEnvironmentITCase(TestExecutionMode mode) {
-		super(mode);
-	}
-
-	@Parameterized.Parameters(name = "Execution mode = {0}")
-	public static Collection<TestExecutionMode[]> executionModes(){
-		Collection<TestExecutionMode[]> c = new ArrayList<TestExecutionMode[]>(1);
-		c.add(new TestExecutionMode[] {TestExecutionMode.CLUSTER});
-		return c;
-	}
-
-
 	/**
 	 * Ensure that the user can pass a custom configuration object to the LocalEnvironment
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
index af51ed6..e976c23 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
@@ -58,7 +58,7 @@ public class IPv6HostnamesITCase extends TestLogger {
 
 		final Inet6Address ipv6address = getLocalIPv6Address();
 		if (ipv6address == null) {
-			System.err.println("--- Cannot find a non-loopback local IPv6 address, skipping IPv6HostnamesITCase");
+			System.err.println("--- Cannot find a non-loopback local IPv6 address that Akka/Netty can bind to; skipping IPv6HostnamesITCase");
 			return;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/BatchScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/BatchScalaAPICompletenessTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/BatchScalaAPICompletenessTest.scala
index a5f1cbb..d50186e 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/BatchScalaAPICompletenessTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/completeness/BatchScalaAPICompletenessTest.scala
@@ -53,7 +53,8 @@ class BatchScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
        "org.apache.flink.api.java.operators.TwoInputOperator.getInput2",
        "org.apache.flink.api.java.operators.TwoInputOperator.getInput1Type",
        "org.apache.flink.api.java.operators.TwoInputOperator.getInput2Type",
-       "org.apache.flink.api.java.ExecutionEnvironment.localExecutionIsAllowed",
+       "org.apache.flink.api.java.ExecutionEnvironment.areExplicitEnvironmentsAllowed",
+       "org.apache.flink.api.java.ExecutionEnvironment.resetContextEnvironment",
        "org.apache.flink.api.java.ExecutionEnvironment.setDefaultLocalParallelism",
 
        // TypeHints are only needed for Java API, Scala API doesn't need them


[4/9] flink git commit: [FLINK-2783] Remove "degreeOfParallelism" API calls

Posted by se...@apache.org.
[FLINK-2783] Remove "degreeOfParallelism" API calls

This closes #1200


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

Branch: refs/heads/master
Commit: f2ea4e4d67968359f7d0594cadac93582b397755
Parents: 82d6236
Author: zentol <s....@web.de>
Authored: Wed Sep 30 11:40:41 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 1 11:03:54 2015 +0200

----------------------------------------------------------------------
 .../flink/api/common/ExecutionConfig.java       | 35 --------------------
 .../flink/api/common/operators/Operator.java    | 25 --------------
 .../flink/api/java/CollectionEnvironment.java   |  9 -----
 .../flink/api/java/ExecutionEnvironment.java    | 35 --------------------
 .../api/java/operators/CoGroupRawOperator.java  |  2 +-
 .../src/main/java/YarnJob.java                  |  2 +-
 .../src/main/java/YarnWordCount.java            |  2 +-
 .../flink/api/scala/ExecutionEnvironment.scala  | 20 -----------
 .../api/java/common/PlanBinder.java             |  6 ++--
 .../spargel/java/record/SpargelIteration.java   |  8 -----
 .../environment/StreamExecutionEnvironment.java | 33 ------------------
 .../api/scala/StreamExecutionEnvironment.scala  | 20 -----------
 .../flink/tez/test/TezProgramTestBase.java      |  2 +-
 13 files changed, 7 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index 3233327..28f3b92 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -206,23 +206,6 @@ public class ExecutionConfig implements Serializable {
 	 *
 	 * @return The parallelism used by operations, unless they override that value. This method
 	 *         returns {@code -1}, if the environment's default parallelism should be used.
-	 * @deprecated Please use {@link #getParallelism}
-	 */
-	@Deprecated
-	public int getDegreeOfParallelism() {
-		return getParallelism();
-	}
-
-	/**
-	 * Gets the parallelism with which operation are executed by default. Operations can
-	 * individually override this value to use a specific parallelism.
-	 *
-	 * Other operations may need to run with a different parallelism - for example calling
-	 * a reduce operation over the entire data set will involve an operation that runs
-	 * with a parallelism of one (the final reduce to the single result value).
-	 *
-	 * @return The parallelism used by operations, unless they override that value. This method
-	 *         returns {@code -1}, if the environment's default parallelism should be used.
 	 */
 	public int getParallelism() {
 		return parallelism;
@@ -239,24 +222,6 @@ public class ExecutionConfig implements Serializable {
 	 * from a JAR file, the default parallelism is the one configured for that setup.
 	 *
 	 * @param parallelism The parallelism to use
-	 * @deprecated Please use {@link #setParallelism}
-	 */
-	@Deprecated
-	public ExecutionConfig setDegreeOfParallelism(int parallelism) {
-		return setParallelism(parallelism);
-	}
-
-	/**
-	 * Sets the parallelism for operations executed through this environment.
-	 * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with
-	 * x parallel instances.
-	 * <p>
-	 * This method overrides the default parallelism for this environment.
-	 * The local execution environment uses by default a value equal to the number of hardware
-	 * contexts (CPU cores / threads). When executing the program via the command line client
-	 * from a JAR file, the default parallelism is the one configured for that setup.
-	 *
-	 * @param parallelism The parallelism to use
 	 */
 	public ExecutionConfig setParallelism(int parallelism) {
 		if (parallelism < 1 && parallelism != -1) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
index 840c253..19294d2 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Operator.java
@@ -163,19 +163,6 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 	 * Gets the parallelism for this contract instance. The parallelism denotes
 	 * how many parallel instances of the user function will be spawned during the execution. If this
 	 * value is <code>-1</code>, then the system will decide the number of parallel instances by itself.
-	 * 
-	 * @return The parallelism.
-	 * @deprecated Please use {@link #getParallelism}
-	 */
-	@Deprecated
-	public int getDegreeOfParallelism() {
-		return getParallelism();
-	}
-
-	/**
-	 * Gets the parallelism for this contract instance. The parallelism denotes
-	 * how many parallel instances of the user function will be spawned during the execution. If this
-	 * value is <code>-1</code>, then the system will decide the number of parallel instances by itself.
 	 *
 	 * @return The parallelism.
 	 */
@@ -187,18 +174,6 @@ public abstract class Operator<OUT> implements Visitable<Operator<?>> {
 	 * Sets the parallelism for this contract instance. The parallelism denotes
 	 * how many parallel instances of the user function will be spawned during the execution. Set this
 	 * value to <code>-1</code> to let the system decide on its own.
-	 * 
-	 * @param parallelism The number of parallel instances to spawn. -1, if unspecified.
-	 * @deprecated Please use {@link #setParallelism}
-	 */
-	@Deprecated
-	public void setDegreeOfParallelism(int parallelism) {
-		setParallelism(parallelism);
-	}
-	/**
-	 * Sets the parallelism for this contract instance. The parallelism denotes
-	 * how many parallel instances of the user function will be spawned during the execution. Set this
-	 * value to <code>-1</code> to let the system decide on its own.
 	 *
 	 * @param parallelism The number of parallel instances to spawn. -1, if unspecified.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
index dbb7cc0..b9e9f81 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java
@@ -34,15 +34,6 @@ public class CollectionEnvironment extends ExecutionEnvironment {
 		return this.lastJobExecutionResult;
 	}
 
-	/**
-	 * @deprecated Please use {@link #getParallelism}
-	 */
-	@Override
-	@Deprecated
-	public int getDegreeOfParallelism() {
-		return getParallelism();
-	}
-
 	@Override
 	public int getParallelism() {
 		return 1; // always serial

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index c69294d..a596765 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -152,23 +152,6 @@ public abstract class ExecutionEnvironment {
 	 * parallelism - for example calling
 	 * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire
 	 * set will insert eventually an operation that runs non-parallel (parallelism of one).
-	 * 
-	 * @return The parallelism used by operations, unless they override that value. This method
-	 *         returns {@code -1}, if the environments default parallelism should be used.
-	 * @deprecated Please use {@link #getParallelism}
-	 */
-	@Deprecated
-	public int getDegreeOfParallelism() {
-		return getParallelism();
-	}
-
-	/**
-	 * Gets the parallelism with which operation are executed by default. Operations can
-	 * individually override this value to use a specific parallelism via
-	 * {@link Operator#setParallelism(int)}. Other operations may need to run with a different
-	 * parallelism - for example calling
-	 * {@link DataSet#reduce(org.apache.flink.api.common.functions.ReduceFunction)} over the entire
-	 * set will insert eventually an operation that runs non-parallel (parallelism of one).
 	 *
 	 * @return The parallelism used by operations, unless they override that value. This method
 	 *         returns {@code -1}, if the environments default parallelism should be used.
@@ -176,24 +159,6 @@ public abstract class ExecutionEnvironment {
 	public int getParallelism() {
 		return config.getParallelism();
 	}
-	
-	/**
-	 * Sets the parallelism for operations executed through this environment.
-	 * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run with
-	 * x parallel instances.
-	 * <p>
-	 * This method overrides the default parallelism for this environment.
-	 * The {@link LocalEnvironment} uses by default a value equal to the number of hardware
-	 * contexts (CPU cores / threads). When executing the program via the command line client 
-	 * from a JAR file, the default parallelism is the one configured for that setup.
-	 * 
-	 * @param parallelism The parallelism
-	 * @deprecated Please use {@link #setParallelism}
-	 */
-	@Deprecated
-	public void setDegreeOfParallelism(int parallelism) {
-		setParallelism(parallelism);
-	}
 
 	/**
 	 * Sets the parallelism for operations executed through this environment.

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java
index 38326bd..30639c3 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java
@@ -102,7 +102,7 @@ public class CoGroupRawOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2,
 			po.setSecondInput(input2);
 
 			// set dop
-			po.setDegreeOfParallelism(this.getParallelism());
+			po.setParallelism(this.getParallelism());
 
 			return po;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnJob.java
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnJob.java b/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnJob.java
index 51627d5..30885f1 100644
--- a/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnJob.java
+++ b/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnJob.java
@@ -42,7 +42,7 @@ public class YarnJob {
 		
 		// To use Tez YARN execution, use
 		final RemoteTezEnvironment env = RemoteTezEnvironment.create();
-		env.setDegreeOfParallelism(8);
+		env.setParallelism(8);
 
 		/**
 		 * Here, you can start creating your execution plan for Flink.

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnWordCount.java
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnWordCount.java b/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnWordCount.java
index e97dc0b..5f9ef74 100644
--- a/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnWordCount.java
+++ b/flink-quickstart/flink-tez-quickstart/src/main/resources/archetype-resources/src/main/java/YarnWordCount.java
@@ -51,7 +51,7 @@ public class YarnWordCount {
 		
 		// set up the execution environment
 		final RemoteTezEnvironment env = RemoteTezEnvironment.create();
-		env.setDegreeOfParallelism(parallelism);
+		env.setParallelism(parallelism);
 
 		// get input data
 		DataSet<String> text = env.readTextFile(textPath);

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
index 85c5410..3427225 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
@@ -82,18 +82,6 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
    * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
    * with x parallel instances. This value can be overridden by specific operations using
    * [[DataSet.setParallelism]].
-   * @deprecated Please use [[setParallelism]]
-   */
-  @deprecated
-  def setDegreeOfParallelism(parallelism: Int): Unit = {
-    setParallelism(parallelism)
-  }
-
-  /**
-   * Sets the parallelism (parallelism) for operations executed through this environment.
-   * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
-   * with x parallel instances. This value can be overridden by specific operations using
-   * [[DataSet.setParallelism]].
    */
   def setParallelism(parallelism: Int): Unit = {
     javaEnv.setParallelism(parallelism)
@@ -102,14 +90,6 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
   /**
    * Returns the default parallelism for this execution environment. Note that this
    * value can be overridden by individual operations using [[DataSet.setParallelism]]
-   * @deprecated Please use [[getParallelism]]
-   */
-  @deprecated
-  def getDegreeOfParallelism = javaEnv.getParallelism
-
-  /**
-   * Returns the default parallelism for this execution environment. Note that this
-   * value can be overridden by individual operations using [[DataSet.setParallelism]]
    */
   def getParallelism = javaEnv.getParallelism
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/PlanBinder.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/PlanBinder.java b/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/PlanBinder.java
index 8ca0405..ca252f8 100644
--- a/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/PlanBinder.java
+++ b/flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/PlanBinder.java
@@ -84,7 +84,7 @@ public abstract class PlanBinder<INFO extends OperationInfo> {
 			switch (Parameters.valueOf(((String) value.getField(0)).toUpperCase())) {
 				case DOP:
 					Integer dop = (Integer) value.getField(1);
-					env.setDegreeOfParallelism(dop);
+					env.setParallelism(dop);
 					break;
 				case MODE:
 					FLINK_HDFS_PATH = (Boolean) value.getField(1) ? "file:/tmp/flink" : "hdfs:/tmp/flink";
@@ -98,8 +98,8 @@ public abstract class PlanBinder<INFO extends OperationInfo> {
 					break;
 			}
 		}
-		if (env.getDegreeOfParallelism() < 0) {
-			env.setDegreeOfParallelism(1);
+		if (env.getParallelism() < 0) {
+			env.setParallelism(1);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
index 8f1839d..7c137c9 100644
--- a/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
+++ b/flink-staging/flink-spargel/src/main/java/org/apache/flink/spargel/java/record/SpargelIteration.java
@@ -128,14 +128,6 @@ public class SpargelIteration {
 		return this.iteration;
 	}
 
-	/**
-	 * @deprecated Please use {@link #setParallelism}
-	 */
-	@Deprecated
-	public void setDegreeOfParallelism(int parallelism) {
-		setParallelism(parallelism);
-	}
-
 	public void setParallelism(int parallelism) {
 		this.iteration.setParallelism(parallelism);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 5537fd4..598d0df 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -157,39 +157,6 @@ public abstract class StreamExecutionEnvironment {
 	 *
 	 * @param parallelism
 	 * 		The parallelism
-	 * @deprecated Please use {@link #setParallelism}
-	 */
-	@Deprecated
-	public StreamExecutionEnvironment setDegreeOfParallelism(int parallelism) {
-		return setParallelism(parallelism);
-	}
-
-	/**
-	 * Gets the parallelism with which operation are executed by default.
-	 * Operations can individually override this value to use a specific
-	 * parallelism.
-	 *
-	 * @return The parallelism used by operations, unless they override that
-	 * value.
-	 * @deprecated Please use {@link #getParallelism}
-	 */
-	@Deprecated
-	public int getDegreeOfParallelism() {
-		return getParallelism();
-	}
-
-	/**
-	 * Sets the parallelism for operations executed through this environment.
-	 * Setting a parallelism of x here will cause all operators (such as map,
-	 * batchReduce) to run with x parallel instances. This method overrides the
-	 * default parallelism for this environment. The
-	 * {@link LocalStreamEnvironment} uses by default a value equal to the
-	 * number of hardware contexts (CPU cores / threads). When executing the
-	 * program via the command line client from a JAR file, the default degree
-	 * of parallelism is the one configured for that setup.
-	 *
-	 * @param parallelism
-	 * 		The parallelism
 	 */
 	public StreamExecutionEnvironment setParallelism(int parallelism) {
 		if (parallelism < 1) {

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index e538435..2474d8c 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -51,26 +51,6 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
    * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
    * with x parallel instances. This value can be overridden by specific operations using
    * [[DataStream#setParallelism(int)]].
-   * @deprecated Please use [[setParallelism]]
-   */
-  @deprecated
-  def setDegreeOfParallelism(degreeOfParallelism: Int): Unit = {
-    javaEnv.setParallelism(degreeOfParallelism)
-  }
-
-  /**
-   * Returns the default parallelism for this execution environment. Note that this
-   * value can be overridden by individual operations using [[DataStream#setParallelism(int)]]
-   * @deprecated Please use [[getParallelism]]
-   */
-  @deprecated
-  def getDegreeOfParallelism = javaEnv.getParallelism
-
-  /**
-   * Sets the parallelism for operations executed through this environment.
-   * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run
-   * with x parallel instances. This value can be overridden by specific operations using
-   * [[DataStream#setParallelism(int)]].
    */
   def setParallelism(parallelism: Int): Unit = {
     javaEnv.setParallelism(parallelism)

http://git-wip-us.apache.org/repos/asf/flink/blob/f2ea4e4d/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java b/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
index e3c6f1b..6186a47 100644
--- a/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
+++ b/flink-staging/flink-tez/src/test/java/org/apache/flink/tez/test/TezProgramTestBase.java
@@ -45,7 +45,7 @@ public abstract class TezProgramTestBase extends AbstractTestBase {
     }
 
 
-    public void setDegreeOfParallelism(int degreeOfParallelism) {
+    public void setParallelism(int degreeOfParallelism) {
         this.degreeOfParallelism = degreeOfParallelism;
     }
 


[5/9] flink git commit: [hotfix] Simplify new windowing API

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
index 3d9605e..01381f9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
@@ -27,7 +27,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
 import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
-import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
 import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -56,7 +56,7 @@ public class EvictingWindowOperatorTest {
 				GlobalWindows.create(),
 				new TupleKeySelector(),
 				new HeapWindowBuffer.Factory<Tuple2<String, Integer>>(),
-				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer(closeCalled)),
+				new ReduceKeyedWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer(closeCalled)),
 				CountTrigger.of(WINDOW_SLIDE),
 				CountEvictor.of(WINDOW_SIZE));
 

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyWindowTranslationTest.java
deleted file mode 100644
index 6f42514..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/PolicyWindowTranslationTest.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.runtime.operators.windowing;
-
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-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.functions.windowing.KeyedWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-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.evictors.TimeEvictor;
-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.windowpolicy.Count;
-import org.apache.flink.streaming.api.windowing.windowpolicy.Delta;
-import org.apache.flink.streaming.api.windowing.windowpolicy.Time;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * These tests verify that the api calls on
- * {@link org.apache.flink.streaming.api.datastream.KeyedWindowDataStream} instantiate
- * the correct window operator.
- */
-public class PolicyWindowTranslationTest extends StreamingMultipleProgramsTestBase {
-
-	/**
-	 * These tests ensure that the fast aligned time windows operator is used if the
-	 * conditions are right.
-	 */
-	@Test
-	public void testFastTimeWindows() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(Time.of(1000, TimeUnit.MILLISECONDS), Time.of(100, TimeUnit.MILLISECONDS))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(Time.of(1000, TimeUnit.MILLISECONDS))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, Window>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(Tuple tuple,
-							Window window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testNonEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(Count.of(200))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof WindowOperator);
-		WindowOperator winOperator1 = (WindowOperator) operator1;
-		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof PurgingTrigger);
-		Assert.assertTrue(((PurgingTrigger)winOperator1.getTriggerTemplate()).getNestedTrigger() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof GlobalWindows);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(Delta.of(15.0, new DeltaFunction<Object>() {
-					@Override
-					public double getDelta(Object oldDataPoint, Object newDataPoint) {
-						return 0;
-					}
-				}))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, Window>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(Tuple tuple,
-							Window window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof WindowOperator);
-		WindowOperator winOperator2 = (WindowOperator) operator2;
-		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof PurgingTrigger);
-		Assert.assertTrue(((PurgingTrigger)winOperator2.getTriggerTemplate()).getNestedTrigger() instanceof DeltaTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof GlobalWindows);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(Time.of(1000, TimeUnit.MICROSECONDS), Count.of(100))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof EvictingWindowOperator);
-		EvictingWindowOperator winOperator1 = (EvictingWindowOperator) operator1;
-		// ensure that the operator sets the current processing time as timestamp
-		Assert.assertTrue(winOperator1.isSetProcessingTime());
-		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof GlobalWindows);
-		Assert.assertTrue(winOperator1.getEvictor() instanceof TimeEvictor);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(Count.of(1000), Delta.of(1.0, new DeltaFunction<Object>() {
-					@Override
-					public double getDelta(Object oldDataPoint, Object newDataPoint) {
-						return 0;
-					}
-				}))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, Window>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(Tuple tuple,
-							Window window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof EvictingWindowOperator);
-		EvictingWindowOperator winOperator2 = (EvictingWindowOperator) operator2;
-		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof DeltaTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof GlobalWindows);
-		Assert.assertTrue(winOperator2.getEvictor() instanceof CountEvictor);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	// ------------------------------------------------------------------------
-	//  UDFs
-	// ------------------------------------------------------------------------
-
-	public static class DummyReducer extends RichReduceFunction<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
-			return value1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
new file mode 100644
index 0000000..d0aa547
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.runtime.operators.windowing;
+
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+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.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * These tests verify that the api calls on
+ * {@link org.apache.flink.streaming.api.datastream.KeyedWindowDataStream} instantiate
+ * the correct window operator.
+ */
+public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase {
+
+	/**
+	 * These tests ensure that the fast aligned time windows operator is used if the
+	 * conditions are right.
+	 */
+	@Test
+	public void testFastTimeWindows() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.keyBy(0)
+				.timeWindow(Time.of(1000, TimeUnit.MILLISECONDS), Time.of(100, TimeUnit.MILLISECONDS))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.keyBy(0)
+				.timeWindow(Time.of(1000, TimeUnit.MILLISECONDS))
+				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(Tuple tuple,
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
+	}
+
+	// ------------------------------------------------------------------------
+	//  UDFs
+	// ------------------------------------------------------------------------
+
+	public static class DummyReducer extends RichReduceFunction<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
+			return value1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerWindowTranslationTest.java
deleted file mode 100644
index 5078c8c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TriggerWindowTranslationTest.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.runtime.operators.windowing;
-
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple;
-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.functions.windowing.KeyedWindowFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
-import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * These tests verify that the api calls on
- * {@link org.apache.flink.streaming.api.datastream.KeyedTriggerWindowDataStream} instantiate
- * the correct window operator.
- */
-public class TriggerWindowTranslationTest extends StreamingMultipleProgramsTestBase {
-
-	/**
-	 * These tests ensure that the fast aligned time windows operator is used if the
-	 * conditions are right.
-	 */
-	@Test
-	public void testFastTimeWindows() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(Tuple tuple,
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testNonEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
-				.trigger(CountTrigger.of(100))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof WindowOperator);
-		WindowOperator winOperator1 = (WindowOperator) operator1;
-		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(TumblingProcessingTimeWindows.of(1000))
-				.trigger(CountTrigger.of(100))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(Tuple tuple,
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof WindowOperator);
-		WindowOperator winOperator2 = (WindowOperator) operator2;
-		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(SlidingProcessingTimeWindows.of(1000, 100))
-				.evictor(CountEvictor.of(100))
-				.reduceWindow(reducer);
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
-		Assert.assertTrue(operator1 instanceof EvictingWindowOperator);
-		EvictingWindowOperator winOperator1 = (EvictingWindowOperator) operator1;
-		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
-		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(TumblingProcessingTimeWindows.of(1000))
-				.trigger(CountTrigger.of(100))
-				.evictor(TimeEvictor.of(100))
-				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void evaluate(Tuple tuple,
-							TimeWindow window,
-							Iterable<Tuple2<String, Integer>> values,
-							Collector<Tuple2<String, Integer>> out) throws Exception {
-
-					}
-				});
-
-		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
-		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
-		Assert.assertTrue(operator2 instanceof EvictingWindowOperator);
-		EvictingWindowOperator winOperator2 = (EvictingWindowOperator) operator2;
-		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
-		Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	// ------------------------------------------------------------------------
-	//  UDFs
-	// ------------------------------------------------------------------------
-
-	public static class DummyReducer extends RichReduceFunction<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
-			return value1;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index 1bfd1d5..3107d51 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -30,7 +30,7 @@ import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
 import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
-import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceKeyedWindowFunction;
 import org.apache.flink.streaming.api.windowing.triggers.ContinuousWatermarkTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
 import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
@@ -76,7 +76,7 @@ public class WindowOperatorTest {
 				SlidingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceKeyedWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -163,7 +163,7 @@ public class WindowOperatorTest {
 				TumblingTimeWindows.of(WINDOW_SIZE),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceKeyedWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
 				WatermarkTrigger.create());
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -246,7 +246,7 @@ public class WindowOperatorTest {
 				GlobalWindows.create(),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceKeyedWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
 				ContinuousWatermarkTrigger.of(WINDOW_SIZE));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
@@ -331,7 +331,7 @@ public class WindowOperatorTest {
 				GlobalWindows.create(),
 				new TupleKeySelector(),
 				windowBufferFactory,
-				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
+				new ReduceKeyedWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
 				PurgingTrigger.of(CountTrigger.of(WINDOW_SIZE)));
 
 		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
new file mode 100644
index 0000000..43e7715
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
@@ -0,0 +1,201 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.runtime.operators.windowing;
+
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+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.functions.windowing.KeyedWindowFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
+import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
+import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * These tests verify that the api calls on
+ * {@link org.apache.flink.streaming.api.datastream.KeyedWindowDataStream} instantiate
+ * the correct window operator.
+ */
+public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
+
+	/**
+	 * These tests ensure that the fast aligned time windows operator is used if the
+	 * conditions are right.
+	 */
+	@Test
+	public void testFastTimeWindows() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.keyBy(0)
+				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.keyBy(0)
+				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(Tuple tuple,
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
+	}
+
+	@Test
+	@SuppressWarnings("rawtypes")
+	public void testNonEvicting() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.keyBy(0)
+				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.trigger(CountTrigger.of(100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof WindowOperator);
+		WindowOperator winOperator1 = (WindowOperator) operator1;
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.keyBy(0)
+				.window(TumblingProcessingTimeWindows.of(1000))
+				.trigger(CountTrigger.of(100))
+				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(Tuple tuple,
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof WindowOperator);
+		WindowOperator winOperator2 = (WindowOperator) operator2;
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+	}
+
+	@Test
+	@SuppressWarnings("rawtypes")
+	public void testEvicting() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+
+		DummyReducer reducer = new DummyReducer();
+
+		DataStream<Tuple2<String, Integer>> window1 = source
+				.keyBy(0)
+				.window(SlidingProcessingTimeWindows.of(1000, 100))
+				.evictor(CountEvictor.of(100))
+				.reduceWindow(reducer);
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+		Assert.assertTrue(operator1 instanceof EvictingWindowOperator);
+		EvictingWindowOperator winOperator1 = (EvictingWindowOperator) operator1;
+		Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
+		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
+		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+
+		DataStream<Tuple2<String, Integer>> window2 = source
+				.keyBy(0)
+				.window(TumblingProcessingTimeWindows.of(1000))
+				.trigger(CountTrigger.of(100))
+				.evictor(TimeEvictor.of(100))
+				.mapWindow(new KeyedWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void evaluate(Tuple tuple,
+							TimeWindow window,
+							Iterable<Tuple2<String, Integer>> values,
+							Collector<Tuple2<String, Integer>> out) throws Exception {
+
+					}
+				});
+
+		OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+		OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+		Assert.assertTrue(operator2 instanceof EvictingWindowOperator);
+		EvictingWindowOperator winOperator2 = (EvictingWindowOperator) operator2;
+		Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
+		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+		Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
+		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+	}
+
+	// ------------------------------------------------------------------------
+	//  UDFs
+	// ------------------------------------------------------------------------
+
+	public static class DummyReducer extends RichReduceFunction<Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<String, Integer> reduce(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception {
+			return value1;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/5623c15b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
index 6cc206a..a5f1e89 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
@@ -27,7 +27,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.KeyedWindowFunction;
-import org.apache.flink.streaming.api.windowing.windowpolicy.Time;
+import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
 
@@ -78,7 +78,7 @@ public class GroupedProcessingTimeWindowExample {
 		
 		stream
 			.keyBy(0)
-			.window(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
+			.timeWindow(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
 			.reduceWindow(new SummingReducer())
 
 			// alternative: use a mapWindow function which does not pre-aggregate


[3/9] flink git commit: [streaming] [storm] Clean up instantiation of mini clusters and test environments.

Posted by se...@apache.org.
[streaming] [storm] Clean up instantiation of mini clusters and test environments.


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

Branch: refs/heads/master
Commit: 82d6236173093b7e035a21360c7b69c67fd6ae62
Parents: 891db5e
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 30 00:12:00 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 1 11:02:39 2015 +0200

----------------------------------------------------------------------
 .../OperatorStatsAccumulatorTest.java           |  11 +-
 .../api/FlinkLocalCluster.java                  |  88 +++++--
 .../excamation/StormExclamationLocal.java       |   9 +-
 .../api/FlinkTestCluster.java                   | 107 --------
 .../stormcompatibility/api/StormTestBase.java   | 117 +++++++++
 .../ExclamationWithStormBoltITCase.java         |   4 +-
 .../ExclamationWithStormSpoutITCase.java        |   5 +-
 .../StormExclamationLocalITCase.java            |   7 +-
 .../wordcount/BoltTokenizerWordCountITCase.java |   4 +-
 .../BoltTokenizerWordCountPojoITCase.java       |   4 +-
 .../BoltTokenizerWordCountWithNamesITCase.java  |   4 +-
 .../wordcount/SpoutSourceWordCountITCase.java   |   4 +-
 .../wordcount/StormWordCountLocalITCase.java    |   7 +-
 .../StormWordCountLocalNamedITCase.java         |   7 +-
 .../flink/api/java/ExecutionEnvironment.java    |   2 +-
 .../runtime/minicluster/FlinkMiniCluster.scala  |   7 +
 .../java/org/apache/flink/tachyon/HDFSTest.java |  26 +-
 .../api/java/ScalaShellRemoteEnvironment.java   |  10 +-
 .../org.apache.flink/api/scala/FlinkILoop.scala |   9 +-
 .../api/environment/LocalStreamEnvironment.java |  78 ++++--
 .../environment/RemoteStreamEnvironment.java    |  52 +++-
 .../environment/StreamExecutionEnvironment.java | 249 +++++++++++--------
 .../flink/streaming/util/ClusterUtil.java       |  96 -------
 .../util/StreamingMultipleProgramsTestBase.java |  29 +--
 .../util/StreamingProgramTestBase.java          |  58 ++---
 .../streaming/util/TestStreamEnvironment.java   | 150 +++--------
 .../TopSpeedWindowingExampleITCase.java         |   2 +-
 .../flink/tez/client/RemoteTezEnvironment.java  |   2 +-
 .../flink/tez/test/TezProgramTestBase.java      |   3 +-
 .../flink/test/util/AbstractTestBase.java       |  33 ++-
 .../flink/test/util/JavaProgramTestBase.java    |   3 +-
 .../flink/test/util/RecordAPITestBase.java      |   8 +-
 .../apache/flink/test/util/TestBaseUtils.java   |   9 +-
 .../ExecutionEnvironmentITCase.java             |  25 +-
 .../flink/test/runtime/IPv6HostnamesITCase.java |   2 +-
 .../BatchScalaAPICompletenessTest.scala         |   3 +-
 36 files changed, 620 insertions(+), 614 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-operator-stats/src/test/java/org/apache/flink/contrib/operatorstatistics/OperatorStatsAccumulatorTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-operator-stats/src/test/java/org/apache/flink/contrib/operatorstatistics/OperatorStatsAccumulatorTest.java b/flink-contrib/flink-operator-stats/src/test/java/org/apache/flink/contrib/operatorstatistics/OperatorStatsAccumulatorTest.java
index b9a8dc2..887c745 100644
--- a/flink-contrib/flink-operator-stats/src/test/java/org/apache/flink/contrib/operatorstatistics/OperatorStatsAccumulatorTest.java
+++ b/flink-contrib/flink-operator-stats/src/test/java/org/apache/flink/contrib/operatorstatistics/OperatorStatsAccumulatorTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.util.Collector;
 import org.junit.Assert;
@@ -36,6 +37,7 @@ import java.io.Serializable;
 import java.util.Map;
 import java.util.Random;
 
+@SuppressWarnings("serial")
 public class OperatorStatsAccumulatorTest extends AbstractTestBase {
 
 	private static final Logger LOG = LoggerFactory.getLogger(OperatorStatsAccumulatorTest.class);
@@ -43,9 +45,9 @@ public class OperatorStatsAccumulatorTest extends AbstractTestBase {
 	private static final String ACCUMULATOR_NAME = "op-stats";
 
 	public OperatorStatsAccumulatorTest(){
-		super(new Configuration());
+		super(new Configuration(), StreamingMode.BATCH_ONLY);
 	}
-
+	
 	public static class StringToInt extends RichFlatMapFunction<String, Tuple1<Integer>> {
 
 		// Is instantiated later since the runtime context is not yet initialized
@@ -81,9 +83,8 @@ public class OperatorStatsAccumulatorTest extends AbstractTestBase {
 			try {
 				intValue = Integer.parseInt(value);
 				localAccumulator.add(intValue);
-				out.collect(new Tuple1(intValue));
-			} catch (NumberFormatException ex) {
-			}
+				out.collect(new Tuple1<>(intValue));
+			} catch (NumberFormatException ignored) {}
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
index b5eda8b..c139201 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
@@ -25,15 +25,40 @@ import backtype.storm.generated.RebalanceOptions;
 import backtype.storm.generated.StormTopology;
 import backtype.storm.generated.SubmitOptions;
 import backtype.storm.generated.TopologyInfo;
-import org.apache.flink.streaming.util.ClusterUtil;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.FlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * {@link FlinkLocalCluster} mimics a Storm {@link LocalCluster}.
  */
 public class FlinkLocalCluster {
 
+	/** The log used by this mini cluster */
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkLocalCluster.class);
+	
+	/** The flink mini cluster on which to execute the programs */
+	private final FlinkMiniCluster flink;
+
+	
+	public FlinkLocalCluster() {
+		this.flink = new LocalFlinkMiniCluster(new Configuration(), true, StreamingMode.STREAMING);
+		this.flink.start();
+	}
+
+	public FlinkLocalCluster(FlinkMiniCluster flink) {
+		this.flink = Objects.requireNonNull(flink);
+	}
+
 	public void submitTopology(final String topologyName, final Map<?, ?> conf, final FlinkTopology topology)
 			throws Exception {
 		this.submitTopologyWithOpts(topologyName, conf, topology, null);
@@ -41,7 +66,10 @@ public class FlinkLocalCluster {
 
 	public void submitTopologyWithOpts(final String topologyName, final Map<?, ?> conf, final FlinkTopology topology,
 			final SubmitOptions submitOpts) throws Exception {
-		ClusterUtil.startOnMiniCluster(topology.getStreamGraph().getJobGraph(topologyName), topology.getNumberOfTasks());
+		
+		LOG.info("Running Storm topology on FlinkLocalCluster");
+		JobGraph jobGraph = topology.getStreamGraph().getJobGraph(topologyName);
+		flink.submitJobDetached(jobGraph);
 	}
 
 	public void killTopology(final String topologyName) {
@@ -60,7 +88,9 @@ public class FlinkLocalCluster {
 	public void rebalance(final String name, final RebalanceOptions options) {
 	}
 
-	public void shutdown() {}
+	public void shutdown() {
+		flink.stop();
+	}
 
 	public String getTopologyConf(final String id) {
 		return null;
@@ -82,31 +112,57 @@ public class FlinkLocalCluster {
 		return null;
 	}
 
+	// ------------------------------------------------------------------------
+	//  Access to default local cluster
+	// ------------------------------------------------------------------------
+	
 	// A different {@link FlinkLocalCluster} to be used for execution of ITCases
-	private static FlinkLocalCluster currentCluster = null;
+	private static LocalClusterFactory currentFactory = new DefaultLocalClusterFactory();
 
 	/**
-	 * Returns a {@link FlinkLocalCluster} that should be used for execution. If no cluster was set by {@link
-	 * #initialize(FlinkLocalCluster)} in advance, a new {@link FlinkLocalCluster} is returned.
+	 * Returns a {@link FlinkLocalCluster} that should be used for execution. If no cluster was set by
+	 * {@link #initialize(LocalClusterFactory)} in advance, a new {@link FlinkLocalCluster} is returned.
 	 *
 	 * @return a {@link FlinkLocalCluster} to be used for execution
 	 */
 	public static FlinkLocalCluster getLocalCluster() {
-		if (currentCluster == null) {
-			currentCluster = new FlinkLocalCluster();
-		}
-
-		return currentCluster;
+		return currentFactory.createLocalCluster();
 	}
 
 	/**
-	 * Sets a different {@link FlinkLocalCluster} to be used for execution.
+	 * Sets a different factory for FlinkLocalClusters to be used for execution.
 	 *
-	 * @param cluster
-	 * 		the {@link FlinkLocalCluster} to be used for execution
+	 * @param clusterFactory
+	 * 		The LocalClusterFactory to create the local clusters for execution.
 	 */
-	public static void initialize(final FlinkLocalCluster cluster) {
-		currentCluster = cluster;
+	public static void initialize(LocalClusterFactory clusterFactory) {
+		currentFactory = Objects.requireNonNull(clusterFactory);
 	}
+	
+	// ------------------------------------------------------------------------
+	//  Cluster factory
+	// ------------------------------------------------------------------------
 
+	/**
+	 * A factory that creates local clusters.
+	 */
+	public static interface LocalClusterFactory {
+
+		/**
+		 * Creates a local flink cluster.
+		 * @return A local flink cluster.
+		 */
+		FlinkLocalCluster createLocalCluster();
+	}
+
+	/**
+	 * A factory that instantiates a FlinkLocalCluster.
+	 */
+	public static class DefaultLocalClusterFactory implements LocalClusterFactory {
+		
+		@Override
+		public FlinkLocalCluster createLocalCluster() {
+			return new FlinkLocalCluster();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java
index 5941ff0..bd1220c 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java
@@ -23,18 +23,19 @@ import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder;
 
 /**
  * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text
- * files in a streaming fashion. The program is constructed as a regular {@link StormTopology} and
- * submitted to Flink for execution in the same way as to a Storm {@link LocalCluster}.
+ * files in a streaming fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology}
+ * and submitted to Flink for execution in the same way as to a Storm {@link backtype.storm.LocalCluster}.
  * <p/>
  * This example shows how to run program directly within Java, thus it cannot be used to submit a
- * {@link StormTopology} via Flink command line clients (ie, bin/flink).
+ * {@link backtype.storm.generated.StormTopology} via Flink command line clients (ie, bin/flink).
  * <p/>
  * <p/>
  * The input is a plain text file with lines separated by newline characters.
  * <p/>
  * <p/>
  * Usage: <code>StormExclamationLocal &lt;text path&gt; &lt;result path&gt;</code><br/>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * If no parameters are provided, the program is run with default data from
+ * {@link org.apache.flink.examples.java.wordcount.util.WordCountData}.
  * <p/>
  * <p/>
  * This example shows how to:

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.java
deleted file mode 100644
index 68f1216..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.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.stormcompatibility.api;
-
-import backtype.storm.LocalCluster;
-import backtype.storm.generated.ClusterSummary;
-import backtype.storm.generated.KillOptions;
-import backtype.storm.generated.RebalanceOptions;
-import backtype.storm.generated.StormTopology;
-import backtype.storm.generated.SubmitOptions;
-import backtype.storm.generated.TopologyInfo;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-
-import java.util.Map;
-
-/**
- * {@link FlinkTestCluster} mimics a Storm {@link LocalCluster} for ITCases via a {@link TestStreamEnvironment}.
- */
-public class FlinkTestCluster extends FlinkLocalCluster {
-
-	@Override
-	public void submitTopology(final String topologyName, final Map<?, ?> conf, final FlinkTopology topology)
-			throws Exception {
-		this.submitTopologyWithOpts(topologyName, conf, topology, null);
-	}
-
-	@Override
-	public void submitTopologyWithOpts(final String topologyName, final Map<?, ?> conf, final FlinkTopology topology,
-			final SubmitOptions submitOpts)
-			throws Exception {
-		final TestStreamEnvironment env = (TestStreamEnvironment) StreamExecutionEnvironment.getExecutionEnvironment();
-		env.start(topology.getStreamGraph().getJobGraph(topologyName));
-	}
-
-	@Override
-	public void killTopology(final String topologyName) {
-	}
-
-	@Override
-	public void killTopologyWithOpts(final String name, final KillOptions options) {
-	}
-
-	@Override
-	public void activate(final String topologyName) {
-	}
-
-	@Override
-	public void deactivate(final String topologyName) {
-	}
-
-	@Override
-	public void rebalance(final String name, final RebalanceOptions options) {
-	}
-
-	@Override
-	public void shutdown() {
-		final TestStreamEnvironment env = (TestStreamEnvironment) StreamExecutionEnvironment.getExecutionEnvironment();
-		try {
-			env.shutdown();
-		} catch (final InterruptedException e) {
-			e.printStackTrace();
-		}
-	}
-
-	@Override
-	public String getTopologyConf(final String id) {
-		return null;
-	}
-
-	@Override
-	public StormTopology getTopology(final String id) {
-		return null;
-	}
-
-	@Override
-	public ClusterSummary getClusterInfo() {
-		return null;
-	}
-
-	@Override
-	public TopologyInfo getTopologyInfo(final String id) {
-		return null;
-	}
-
-	@Override
-	public Map<?, ?> getState() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java
new file mode 100644
index 0000000..dd6d0d9
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.stormcompatibility.api;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Base class for Storm tests.
+ */
+public abstract class StormTestBase extends AbstractTestBase {
+	
+	public static final int DEFAULT_PARALLELISM = 4;
+	
+	public StormTestBase() {
+		this(new Configuration());
+	}
+	
+	public StormTestBase(Configuration config) {
+		super(config, StreamingMode.STREAMING);
+		setTaskManagerNumSlots(DEFAULT_PARALLELISM);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Methods to create the test program and for pre- and post- test work
+	// ------------------------------------------------------------------------
+
+	protected abstract void testProgram() throws Exception;
+
+	protected void preSubmit() throws Exception {}
+
+	protected void postSubmit() throws Exception {}
+
+	// ------------------------------------------------------------------------
+	//  Test entry point
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testJob() throws Exception {
+		try {
+			// pre-submit
+			try {
+				preSubmit();
+			}
+			catch (Exception e) {
+				System.err.println(e.getMessage());
+				e.printStackTrace();
+				fail("Pre-submit work caused an error: " + e.getMessage());
+			}
+
+			// prepare the test environment
+			startCluster();
+
+			// we need to initialize the stream test environment, and the storm local cluster
+			TestStreamEnvironment.setAsContext(this.executor, DEFAULT_PARALLELISM);
+			
+			FlinkLocalCluster.initialize(new FlinkLocalCluster.LocalClusterFactory() {
+				@Override
+				public FlinkLocalCluster createLocalCluster() {
+					return new FlinkLocalCluster(executor);
+				}
+			});
+
+			// call the test program
+			try {
+				testProgram();
+			}
+			catch (Exception e) {
+				System.err.println(e.getMessage());
+				e.printStackTrace();
+				fail("Error while calling the test program: " + e.getMessage());
+			}
+
+			// post-submit
+			try {
+				postSubmit();
+			}
+			catch (Exception e) {
+				System.err.println(e.getMessage());
+				e.printStackTrace();
+				fail("Post-submit work caused an error: " + e.getMessage());
+			}
+		}
+		finally {
+			// reset the FlinkLocalCluster to its default behavior
+			FlinkLocalCluster.initialize(new FlinkLocalCluster.DefaultLocalClusterFactory());
+			
+			// reset the StreamExecutionEnvironment to its default behavior
+			TestStreamEnvironment.unsetAsContext();
+			
+			// clean up all resources
+			stopCluster();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java
index 930f87b..f47a58f 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.stormcompatibility.exclamation;
 
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.stormcompatibility.excamation.ExclamationWithStormBolt;
 import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class ExclamationWithStormBoltITCase extends StreamingProgramTestBase {
+public class ExclamationWithStormBoltITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java
index 4c515ce..2a8ac24 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.stormcompatibility.exclamation;
 
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.stormcompatibility.excamation.ExclamationWithStormSpout;
 import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class ExclamationWithStormSpoutITCase extends StreamingProgramTestBase {
+public class ExclamationWithStormSpoutITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;
@@ -43,5 +43,4 @@ public class ExclamationWithStormSpoutITCase extends StreamingProgramTestBase {
 	protected void testProgram() throws Exception {
 		ExclamationWithStormSpout.main(new String[]{this.textPath, this.resultPath});
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java
index d6bcf30..6cba39a 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java
@@ -18,21 +18,18 @@
 
 package org.apache.flink.stormcompatibility.exclamation;
 
-import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
-import org.apache.flink.stormcompatibility.api.FlinkTestCluster;
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.stormcompatibility.excamation.StormExclamationLocal;
 import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class StormExclamationLocalITCase extends StreamingProgramTestBase {
+public class StormExclamationLocalITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;
 
 	@Override
 	protected void preSubmit() throws Exception {
-		FlinkLocalCluster.initialize(new FlinkTestCluster());
 		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
 		this.resultPath = this.getTempDirPath("result");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
index 9228474..c9516ff 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.stormcompatibility.wordcount;
 
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class BoltTokenizerWordCountITCase extends StreamingProgramTestBase {
+public class BoltTokenizerWordCountITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java
index dc75c25..351014e 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.stormcompatibility.wordcount;
 
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class BoltTokenizerWordCountPojoITCase extends StreamingProgramTestBase {
+public class BoltTokenizerWordCountPojoITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java
index e147f53..c2ed088 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.stormcompatibility.wordcount;
 
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class BoltTokenizerWordCountWithNamesITCase extends StreamingProgramTestBase {
+public class BoltTokenizerWordCountWithNamesITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
index 9d7b869..93361c5 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.stormcompatibility.wordcount;
 
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class SpoutSourceWordCountITCase extends StreamingProgramTestBase {
+public class SpoutSourceWordCountITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
index 2427818..6b51cbd 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
@@ -18,19 +18,16 @@
 
 package org.apache.flink.stormcompatibility.wordcount;
 
-import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
-import org.apache.flink.stormcompatibility.api.FlinkTestCluster;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class StormWordCountLocalITCase extends StreamingProgramTestBase {
+public class StormWordCountLocalITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;
 
 	@Override
 	protected void preSubmit() throws Exception {
-		FlinkLocalCluster.initialize(new FlinkTestCluster());
 		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
 		this.resultPath = this.getTempDirPath("result");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java
index 8b9a729..a9e9884 100644
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java
@@ -18,19 +18,16 @@
 
 package org.apache.flink.stormcompatibility.wordcount;
 
-import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
-import org.apache.flink.stormcompatibility.api.FlinkTestCluster;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.stormcompatibility.api.StormTestBase;
 import org.apache.flink.test.testdata.WordCountData;
 
-public class StormWordCountLocalNamedITCase extends StreamingProgramTestBase {
+public class StormWordCountLocalNamedITCase extends StormTestBase {
 
 	protected String textPath;
 	protected String resultPath;
 
 	@Override
 	protected void preSubmit() throws Exception {
-		FlinkLocalCluster.initialize(new FlinkTestCluster());
 		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
 		this.resultPath = this.getTempDirPath("result");
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index 084e608..c69294d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -75,7 +75,7 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 
 /**
- * The ExecutionEnviroment is the context in which a program is executed. A
+ * The ExecutionEnvironment is the context in which a program is executed. A
  * {@link LocalEnvironment} will cause execution in the current JVM, a
  * {@link RemoteEnvironment} will cause execution on a remote setup.
  * <p>

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
index b3cff51..77e977f 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
@@ -112,6 +112,8 @@ abstract class FlinkMiniCluster(
   var taskManagerActors: Option[Seq[ActorRef]] = None
 
   protected var leaderRetrievalService: Option[LeaderRetrievalService] = None
+  
+  private var isRunning = false
 
   // --------------------------------------------------------------------------
   //                           Abstract Methods
@@ -271,6 +273,8 @@ abstract class FlinkMiniCluster(
     if(waitForTaskManagerRegistration) {
       waitForTaskManagersToBeRegistered()
     }
+
+    isRunning = true
   }
 
   def startWebServer(
@@ -314,6 +318,7 @@ abstract class FlinkMiniCluster(
     awaitTermination()
 
     leaderRetrievalService.foreach(_.stop())
+    isRunning = false
   }
 
   protected def shutdown(): Unit = {
@@ -354,6 +359,8 @@ abstract class FlinkMiniCluster(
       _ foreach(_.awaitTermination())
     }
   }
+  
+  def running = isRunning
 
   // --------------------------------------------------------------------------
   //                          Utility Methods

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java
index 633d022..5ec0add 100644
--- a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java
+++ b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java
@@ -99,14 +99,21 @@ public class HDFSTest {
 		try {
 			FileSystem fs = file.getFileSystem();
 			Assert.assertTrue("Must be HadoopFileSystem", fs instanceof HadoopFileSystem);
-			new DopOneTestEnvironment();
+			
+			DopOneTestEnvironment.setAsContext();
 			try {
 				WordCount.main(new String[]{file.toString(), result.toString()});
-			} catch(Throwable t) {
+			}
+			catch(Throwable t) {
 				t.printStackTrace();
 				Assert.fail("Test failed with " + t.getMessage());
 			}
+			finally {
+				DopOneTestEnvironment.unsetAsContext();
+			}
+			
 			Assert.assertTrue("No result file present", hdfs.exists(result));
+			
 			// validate output:
 			org.apache.hadoop.fs.FSDataInputStream inStream = hdfs.open(result);
 			StringWriter writer = new StringWriter();
@@ -159,16 +166,23 @@ public class HDFSTest {
 	}
 
 	// package visible
-	static final class DopOneTestEnvironment extends LocalEnvironment {
-		static {
+	static abstract class DopOneTestEnvironment extends ExecutionEnvironment {
+		
+		public static void setAsContext() {
+			final LocalEnvironment le = new LocalEnvironment();
+			le.setParallelism(1);
+
 			initializeContextEnvironment(new ExecutionEnvironmentFactory() {
+
 				@Override
 				public ExecutionEnvironment createExecutionEnvironment() {
-					LocalEnvironment le = new LocalEnvironment();
-					le.setParallelism(1);
 					return le;
 				}
 			});
 		}
+		
+		public static void unsetAsContext() {
+			resetContextEnvironment();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java b/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java
index 859c686..cb37fd4 100644
--- a/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java
+++ b/flink-staging/flink-scala-shell/src/main/java/org/apache/flink/api/java/ScalaShellRemoteEnvironment.java
@@ -85,7 +85,11 @@ public class ScalaShellRemoteEnvironment extends RemoteEnvironment {
 		return executor.executePlan(p);
 	}
 
-	public void setAsContext() {
+	public static void disableAllContextAndOtherEnvironments() {
+		
+		// we create a context environment that prevents the instantiation of further
+		// context environments. at the same time, setting the context environment prevents manual
+		// creation of local and remote environments
 		ExecutionEnvironmentFactory factory = new ExecutionEnvironmentFactory() {
 			@Override
 			public ExecutionEnvironment createExecutionEnvironment() {
@@ -95,4 +99,8 @@ public class ScalaShellRemoteEnvironment extends RemoteEnvironment {
 		};
 		initializeContextEnvironment(factory);
 	}
+	
+	public static void resetContextEnvironments() {
+		ExecutionEnvironment.resetContextEnvironment();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkILoop.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkILoop.scala b/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkILoop.scala
index 1e96ba3..cd8a846 100644
--- a/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkILoop.scala
+++ b/flink-staging/flink-scala-shell/src/main/scala/org.apache.flink/api/scala/FlinkILoop.scala
@@ -53,8 +53,15 @@ class FlinkILoop(
   }
   // remote environment
   private val remoteEnv: ScalaShellRemoteEnvironment = {
+    // allow creation of environments
+    ScalaShellRemoteEnvironment.resetContextEnvironments()
+    
+    // create our environment that submits against the cluster (local or remote)
     val remoteEnv = new ScalaShellRemoteEnvironment(host, port, this)
-    remoteEnv.setAsContext()
+    
+    // prevent further instantiation of environments
+    ScalaShellRemoteEnvironment.disableAllContextAndOtherEnvironments()
+    
     remoteEnv
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
index 4c002d1..f0bd174 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java
@@ -17,30 +17,57 @@
 
 package org.apache.flink.streaming.api.environment;
 
+import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.util.ClusterUtil;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The LocalStreamEnvironment is a StreamExecutionEnvironment that runs the program locally,
+ * multi-threaded, in the JVM where the environment is instantiated. It spawns an embedded
+ * Flink cluster in the background and executes the program on that cluster.
+ *
+ * <p>When this environment is instantiated, it uses a default parallelism of {@code 1}. The default
+ * parallelism can be set via {@link #setParallelism(int)}.
+ *
+ * <p>Local environments can also be instantiated through {@link StreamExecutionEnvironment#createLocalEnvironment()}
+ * and {@link StreamExecutionEnvironment#createLocalEnvironment(int)}. The former version will pick a
+ * default parallelism equal to the number of hardware contexts in the local machine.
+ */
 public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 
+	private static final Logger LOG = LoggerFactory.getLogger(LocalStreamEnvironment.class);
+	
+	/** The configuration to use for the local cluster */
 	private final Configuration conf;
 
+	/**
+	 * Creates a new local stream environment that uses the default configuration.
+	 */
 	public LocalStreamEnvironment() {
-		conf = null;
-	}
-	public LocalStreamEnvironment(Configuration conf) {
-		this.conf = conf;
+		this(null);
 	}
 
 	/**
-	 * Executes the JobGraph of the on a mini cluster of CLusterUtil with a
-	 * default name.
+	 * Creates a new local stream environment that configures its local executor with the given configuration.
 	 *
-	 * @return The result of the job execution, containing elapsed time and accumulators.
+	 * @param config The configuration used to configure the local executor.
 	 */
-	@Override
-	public JobExecutionResult execute() throws Exception {
-		return execute(DEFAULT_JOB_NAME);
+	public LocalStreamEnvironment(Configuration config) {
+		if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) {
+			throw new InvalidProgramException(
+					"The LocalStreamEnvironment cannot be used when submitting a program through a client, " +
+							"or running in a TestEnvironment context.");
+		}
+		
+		this.conf = config == null ? new Configuration() : config;
 	}
 
 	/**
@@ -53,9 +80,30 @@ public class LocalStreamEnvironment extends StreamExecutionEnvironment {
 	 */
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		JobExecutionResult result = ClusterUtil.runOnMiniCluster(getStreamGraph().getJobGraph(),
-				getParallelism(), -1, getConfig().isSysoutLoggingEnabled(), false, this.conf);
-		transformations.clear();
-		return result;
+		// transform the streaming program into a JobGraph
+		JobGraph jobGraph = getStreamGraph().getJobGraph(jobName);
+		
+		Configuration configuration = new Configuration();
+		configuration.addAll(jobGraph.getJobConfiguration());
+
+		configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
+		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, getParallelism());
+		
+		// add (and override) the settings with what the user defined
+		configuration.addAll(this.conf);
+		
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Running job on local embedded Flink mini cluster");
+		}
+
+		LocalFlinkMiniCluster exec = new LocalFlinkMiniCluster(configuration, true, StreamingMode.STREAMING);
+		try {
+			exec.start();
+			return exec.submitJobAndWait(jobGraph, getConfig().isSysoutLoggingEnabled());
+		}
+		finally {
+			transformations.clear();
+			exec.stop();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
index 29439f6..ccf51d2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
@@ -22,7 +22,9 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.client.program.Client;
 import org.apache.flink.client.program.JobWithJars;
 import org.apache.flink.client.program.ProgramInvocationException;
@@ -41,6 +43,9 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 	private final String host;
 	private final int port;
 	private final List<File> jarFiles;
+	
+	/** The configuration used to parametrize the client that connects to the remote cluster */
+	private final Configuration config;
 
 	/**
 	 * Creates a new RemoteStreamEnvironment that points to the master
@@ -59,17 +64,46 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 	 *            provided in the JAR files.
 	 */
 	public RemoteStreamEnvironment(String host, int port, String... jarFiles) {
+		this(host, port, null, jarFiles);
+	}
+
+	/**
+	 * Creates a new RemoteStreamEnvironment that points to the master
+	 * (JobManager) described by the given host name and port.
+	 *
+	 * @param host
+	 *            The host name or address of the master (JobManager), where the
+	 *            program should be executed.
+	 * @param port
+	 *            The port of the master (JobManager), where the program should
+	 *            be executed.
+	 * @param config
+	 *            The configuration used to parametrize the client that connects to the
+	 *            remote cluster.
+	 * @param jarFiles
+	 *            The JAR files with code that needs to be shipped to the
+	 *            cluster. If the program uses user-defined functions,
+	 *            user-defined input formats, or any libraries, those must be
+	 *            provided in the JAR files.
+	 */
+	public RemoteStreamEnvironment(String host, int port, Configuration config, String... jarFiles) {
+		if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) {
+			throw new InvalidProgramException(
+					"The RemoteEnvironment cannot be used when submitting a program through a client, " +
+							"or running in a TestEnvironment context.");
+		}
+		
 		if (host == null) {
 			throw new NullPointerException("Host must not be null.");
 		}
-
 		if (port < 1 || port >= 0xffff) {
 			throw new IllegalArgumentException("Port out of range");
 		}
 
 		this.host = host;
 		this.port = port;
-		this.jarFiles = new ArrayList<File>();
+		this.config = config == null ? new Configuration() : config;
+		this.jarFiles = new ArrayList<File>(jarFiles.length);
 		for (String jarFile : jarFiles) {
 			File file = new File(jarFile);
 			try {
@@ -83,13 +117,6 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 	}
 
 	@Override
-	public JobExecutionResult execute() throws ProgramInvocationException {
-		JobGraph jobGraph = getStreamGraph().getJobGraph();
-		transformations.clear();
-		return executeRemotely(jobGraph);
-	}
-
-	@Override
 	public JobExecutionResult execute(String jobName) throws ProgramInvocationException {
 		JobGraph jobGraph = getStreamGraph().getJobGraph(jobName);
 		transformations.clear();
@@ -112,9 +139,12 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
 			jobGraph.addJar(new Path(file.getAbsolutePath()));
 		}
 
-		Configuration configuration = jobGraph.getJobConfiguration();
 		ClassLoader usercodeClassLoader = JobWithJars.buildUserCodeClassLoader(jarFiles, getClass().getClassLoader());
-
+		
+		Configuration configuration = new Configuration();
+		configuration.addAll(jobGraph.getJobConfiguration());
+		configuration.addAll(this.config);
+		
 		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, host);
 		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 2a31390..5537fd4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -80,26 +80,46 @@ import java.util.List;
 import java.util.Objects;
 
 /**
- * {@link org.apache.flink.api.java.ExecutionEnvironment} for streaming jobs. An instance of it is
+ * An ExecutionEnvironment for streaming jobs. An instance of it is
  * necessary to construct streaming topologies.
  */
+/**
+ * The StreamExecutionEnvironment is the context in which a streaming program is executed. A
+ * {@link LocalStreamEnvironment} will cause execution in the current JVM, a
+ * {@link RemoteStreamEnvironment} will cause execution on a remote setup.
+ * 
+ * <p>The environment provides methods to control the job execution (such as setting the parallelism
+ * or the fault tolerance/checkpointing parameters) and to interact with the outside world (data access).
+ *
+ * @see org.apache.flink.streaming.api.environment.LocalStreamEnvironment
+ * @see org.apache.flink.streaming.api.environment.RemoteStreamEnvironment
+ */
 public abstract class StreamExecutionEnvironment {
 
+	/** The default name to use for a streaming job if no other name has been specified */
 	public static final String DEFAULT_JOB_NAME = "Flink Streaming Job";
-
-	private static int defaultLocalParallelism = Runtime.getRuntime().availableProcessors();
 	
 	/** The time characteristic that is used if none other is set */
-	private static TimeCharacteristic DEFAULT_TIME_CHARACTERISTIC = TimeCharacteristic.ProcessingTime;
+	private static final TimeCharacteristic DEFAULT_TIME_CHARACTERISTIC = TimeCharacteristic.ProcessingTime;
+
+	/** The default buffer timeout (max delay of records in the network stack) */
+	private static final long DEFAULT_NETWORK_BUFFER_TIMEOUT = 100L;
+
+	/** The environment of the context (local by default, cluster if invoked through command line) */
+	private static StreamExecutionEnvironmentFactory contextEnvironmentFactory;
+
+	/** The default parallelism used when creating a local environment */
+	private static int defaultLocalParallelism = Runtime.getRuntime().availableProcessors();
 	
 	// ------------------------------------------------------------------------
-	
-	private long bufferTimeout = 100;
 
+	/** The execution configuration for this environment */
 	private final ExecutionConfig config = new ExecutionConfig();
-
+	
 	protected final List<StreamTransformation<?>> transformations = new ArrayList<>();
-
+	
+	private long bufferTimeout = DEFAULT_NETWORK_BUFFER_TIMEOUT;
+	
 	protected boolean isChainingEnabled = true;
 
 	protected long checkpointInterval = -1; // disabled
@@ -113,9 +133,7 @@ public abstract class StreamExecutionEnvironment {
 	/** The time characteristic used by the data streams */
 	private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC;
 
-	/** The environment of the context (local by default, cluster if invoked through command line) */
-	private static StreamExecutionEnvironmentFactory contextEnvironmentFactory;
-
+	
 	// --------------------------------------------------------------------------------------------
 	// Constructor and Properties
 	// --------------------------------------------------------------------------------------------
@@ -1157,8 +1175,90 @@ public abstract class StreamExecutionEnvironment {
 		return new DataStreamSource<OUT>(this, typeInfo, sourceOperator, isParallel, sourceName);
 	}
 
+	/**
+	 * Triggers the program execution. The environment will execute all parts of
+	 * the program that have resulted in a "sink" operation. Sink operations are
+	 * for example printing results or forwarding them to a message queue.
+	 * <p/>
+	 * The program execution will be logged and displayed with a generated
+	 * default name.
+	 *
+	 * @return The result of the job execution, containing elapsed time and accumulators.
+	 * @throws Exception which occurs during job execution.
+	 */
+	public JobExecutionResult execute() throws Exception {
+		return execute(DEFAULT_JOB_NAME);
+	}
+
+	/**
+	 * Triggers the program execution. The environment will execute all parts of
+	 * the program that have resulted in a "sink" operation. Sink operations are
+	 * for example printing results or forwarding them to a message queue.
+	 * <p/>
+	 * The program execution will be logged and displayed with the provided name
+	 *
+	 * @param jobName
+	 * 		Desired name of the job
+	 * @return The result of the job execution, containing elapsed time and accumulators.
+	 * @throws Exception which occurs during job execution.
+	 */
+	public abstract JobExecutionResult execute(String jobName) throws Exception;
+
+	/**
+	 * Getter of the {@link org.apache.flink.streaming.api.graph.StreamGraph} of the streaming job.
+	 *
+	 * @return The streamgraph representing the transformations
+	 */
+	public StreamGraph getStreamGraph() {
+		if (transformations.size() <= 0) {
+			throw new IllegalStateException("No operators defined in streaming topology. Cannot execute.");
+		}
+		return StreamGraphGenerator.generate(this, transformations);
+	}
+
+	/**
+	 * Creates the plan with which the system will execute the program, and
+	 * returns it as a String using a JSON representation of the execution data
+	 * flow graph. Note that this needs to be called, before the plan is
+	 * executed.
+	 *
+	 * @return The execution plan of the program, as a JSON String.
+	 */
+	public String getExecutionPlan() {
+		return getStreamGraph().getStreamingPlanAsJSON();
+	}
+
+	/**
+	 * 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}
+	 */
+	public <F> F clean(F f) {
+		if (getConfig().isClosureCleanerEnabled()) {
+			ClosureCleaner.clean(f, true);
+		}
+		ClosureCleaner.ensureSerializable(f);
+		return f;
+	}
+
+	/**
+	 * Adds an operator to the list of operators that should be executed when calling
+	 * {@link #execute}.
+	 *
+	 * <p>
+	 * When calling {@link #execute()} only the operators that where previously added to the list
+	 * are executed.
+	 *
+	 * <p>
+	 * This is not meant to be used by users. The API methods that create operators must call
+	 * this method.
+	 */
+	public void addOperator(StreamTransformation<?> transformation) {
+		Preconditions.checkNotNull(transformation, "transformation must not be null.");
+		this.transformations.add(transformation);
+	}
+
 	// --------------------------------------------------------------------------------------------
-	// Instantiation of Execution Contexts
+	//  Factory methods for ExecutionEnvironments
 	// --------------------------------------------------------------------------------------------
 
 	/**
@@ -1175,6 +1275,10 @@ public abstract class StreamExecutionEnvironment {
 			return contextEnvironmentFactory.createExecutionEnvironment();
 		}
 
+		// because the streaming project depends on "flink-clients" (and not the other way around)
+		// we currently need to intercept the data set environment and create a dependent stream env.
+		// this should be fixed once we rework the project dependencies
+		
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		if (env instanceof ContextEnvironment) {
 			ContextEnvironment ctx = (ContextEnvironment) env;
@@ -1187,8 +1291,9 @@ public abstract class StreamExecutionEnvironment {
 		}
 	}
 
-	private static StreamExecutionEnvironment createContextEnvironment(Client client,
-			List<File> jars, int parallelism, boolean wait) {
+	private static StreamExecutionEnvironment createContextEnvironment(
+			Client client, List<File> jars, int parallelism, boolean wait)
+	{
 		return new StreamContextEnvironment(client, jars, parallelism, wait);
 	}
 
@@ -1236,11 +1341,9 @@ public abstract class StreamExecutionEnvironment {
 	public static LocalStreamEnvironment createLocalEnvironment(int parallelism, Configuration configuration) {
 		LocalStreamEnvironment currentEnvironment = new LocalStreamEnvironment(configuration);
 		currentEnvironment.setParallelism(parallelism);
-		return (LocalStreamEnvironment) currentEnvironment;
+		return currentEnvironment;
 	}
 
-	// TODO:fix cluster default parallelism
-
 	/**
 	 * Creates a {@link RemoteStreamEnvironment}. The remote environment sends
 	 * (parts of) the program to a cluster for execution. Note that all file
@@ -1261,8 +1364,8 @@ public abstract class StreamExecutionEnvironment {
 	 * 		provided in the JAR files.
 	 * @return A remote environment that executes the program on a cluster.
 	 */
-	public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
-			String... jarFiles) {
+	public static StreamExecutionEnvironment createRemoteEnvironment(
+			String host, int port, String... jarFiles) {
 		return new RemoteStreamEnvironment(host, port, jarFiles);
 	}
 
@@ -1287,93 +1390,41 @@ public abstract class StreamExecutionEnvironment {
 	 * 		provided in the JAR files.
 	 * @return A remote environment that executes the program on a cluster.
 	 */
-	public static StreamExecutionEnvironment createRemoteEnvironment(String host, int port,
-			int parallelism, String... jarFiles) {
+	public static StreamExecutionEnvironment createRemoteEnvironment(
+			String host, int port, int parallelism, String... jarFiles)
+	{
 		RemoteStreamEnvironment env = new RemoteStreamEnvironment(host, port, jarFiles);
 		env.setParallelism(parallelism);
 		return env;
 	}
 
 	/**
-	 * Triggers the program execution. The environment will execute all parts of
-	 * the program that have resulted in a "sink" operation. Sink operations are
-	 * for example printing results or forwarding them to a message queue.
-	 * <p/>
-	 * The program execution will be logged and displayed with a generated
-	 * default name.
-	 *
-	 * @return The result of the job execution, containing elapsed time and accumulators.
-	 * @throws Exception which occurs during job execution.
-	 */
-	public abstract JobExecutionResult execute() throws Exception;
-
-	/**
-	 * Triggers the program execution. The environment will execute all parts of
-	 * the program that have resulted in a "sink" operation. Sink operations are
-	 * for example printing results or forwarding them to a message queue.
-	 * <p/>
-	 * The program execution will be logged and displayed with the provided name
-	 *
-	 * @param jobName
-	 * 		Desired name of the job
-	 * @return The result of the job execution, containing elapsed time and accumulators.
-	 * @throws Exception which occurs during job execution.
-	 */
-	public abstract JobExecutionResult execute(String jobName) throws Exception;
-
-	/**
-	 * Getter of the {@link org.apache.flink.streaming.api.graph.StreamGraph} of the streaming job.
-	 *
-	 * @return The streamgraph representing the transformations
-	 */
-	public StreamGraph getStreamGraph() {
-		if (transformations.size() <= 0) {
-			throw new IllegalStateException("No operators defined in streaming topology. Cannot execute.");
-		}
-		return StreamGraphGenerator.generate(this, transformations);
-	}
-
-	/**
-	 * Creates the plan with which the system will execute the program, and
-	 * returns it as a String using a JSON representation of the execution data
-	 * flow graph. Note that this needs to be called, before the plan is
-	 * executed.
-	 *
-	 * @return The execution plan of the program, as a JSON String.
-	 */
-	public String getExecutionPlan() {
-		return getStreamGraph().getStreamingPlanAsJSON();
-	}
-
-	/**
-	 * 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}
-	 */
-	public <F> F clean(F f) {
-		if (getConfig().isClosureCleanerEnabled()) {
-			ClosureCleaner.clean(f, true);
-		}
-		ClosureCleaner.ensureSerializable(f);
-		return f;
-	}
-
-	/**
-	 * Adds an operator to the list of operators that should be executed when calling
-	 * {@link #execute}.
-	 *
-	 * <p>
-	 * When calling {@link #execute()} only the operators that where previously added to the list
-	 * are executed.
+	 * Creates a {@link RemoteStreamEnvironment}. The remote environment sends
+	 * (parts of) the program to a cluster for execution. Note that all file
+	 * paths used in the program must be accessible from the cluster. The
+	 * execution will use the specified parallelism.
 	 *
-	 * <p>
-	 * This is not meant to be used by users. The API methods that create operators must call
-	 * this method.
+	 * @param host
+	 * 		The host name or address of the master (JobManager), where the
+	 * 		program should be executed.
+	 * @param port
+	 * 		The port of the master (JobManager), where the program should
+	 * 		be executed.
+	 * @param clientConfig
+	 * 		The configuration used by the client that connects to the remote cluster.
+	 * @param jarFiles
+	 * 		The JAR files with code that needs to be shipped to the
+	 * 		cluster. If the program uses user-defined functions,
+	 * 		user-defined input formats, or any libraries, those must be
+	 * 		provided in the JAR files.
+	 * @return A remote environment that executes the program on a cluster.
 	 */
-	public void addOperator(StreamTransformation<?> transformation) {
-		Preconditions.checkNotNull(transformation, "Sinks must not be null.");
-		this.transformations.add(transformation);
+	public static StreamExecutionEnvironment createRemoteEnvironment(
+			String host, int port, Configuration clientConfig, String... jarFiles)
+	{
+		return new RemoteStreamEnvironment(host, port, clientConfig, jarFiles);
 	}
-
+	
 	// --------------------------------------------------------------------------------------------
 	//  Methods to control the context and local environments for execution from packaged programs
 	// --------------------------------------------------------------------------------------------
@@ -1381,4 +1432,8 @@ public abstract class StreamExecutionEnvironment {
 	protected static void initializeContextEnvironment(StreamExecutionEnvironmentFactory ctx) {
 		contextEnvironmentFactory = ctx;
 	}
+	
+	protected static void resetContextEnvironment() {
+		contextEnvironmentFactory = null;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.java
deleted file mode 100644
index d4569fe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/ClusterUtil.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.util;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utility class to manage mini cluster for Apache Flink.
- */
-public final class ClusterUtil {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(ClusterUtil.class);
-
-	/**
-	 * Executes the given JobGraph locally, on a FlinkMiniCluster
-	 * 
-	 * @param jobGraph
-	 *            jobGraph
-	 * @param parallelism
-	 *            numberOfTaskTrackers
-	 * @param memorySize
-	 *            memorySize
-	 * @param customConf
-	 * 		Custom configuration for the LocalExecutor. Can be null.
-	 * @return The result of the job execution, containing elapsed time and accumulators.
-	 */
-	public static JobExecutionResult runOnMiniCluster(JobGraph jobGraph, int parallelism, long memorySize,
-													boolean printDuringExecution, boolean detached, Configuration customConf)
-			throws Exception {
-
-		Configuration configuration = jobGraph.getJobConfiguration();
-
-		LocalFlinkMiniCluster exec = null;
-
-		configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize);
-		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism);
-		if(customConf != null) {
-			configuration.addAll(customConf);
-		}
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Running on mini cluster");
-		}
-
-		try {
-			exec = new LocalFlinkMiniCluster(configuration, true);
-			exec.start();
-			
-			if (detached) {
-				exec.submitJobDetached(jobGraph);
-				return null;
-			} else {
-				return exec.submitJobAndWait(jobGraph, printDuringExecution);
-			}
-		} finally {
-			if (exec != null && !detached) {
-				exec.stop();
-			}
-		}
-	}
-
-	/**
-	 * Start a job in a detached mode on a local mini cluster.
-	 */
-	public static void startOnMiniCluster(JobGraph jobGraph, int parallelism) throws Exception {
-		runOnMiniCluster(jobGraph, parallelism, -1, true, true, null);
-	}
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private ClusterUtil() {
-		throw new RuntimeException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
index d251a5d..4e02f2c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
@@ -35,23 +35,22 @@ import org.junit.BeforeClass;
  * one or more regular test methods and retrieve the StreamExecutionEnvironment from
  * the context:
  *
- * <pre>{@code
- *
- *   @Test
+ * <pre>
+ *   {@literal @}Test
  *   public void someTest() {
  *       StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  *       // test code
  *       env.execute();
  *   }
  *
- *   @Test
+ *   {@literal @}Test
  *   public void anotherTest() {
  *       StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  *       // test code
  *       env.execute();
  *   }
  *
- * }</pre>
+ * </pre>
  */
 public class StreamingMultipleProgramsTestBase extends TestBaseUtils {
 
@@ -61,32 +60,22 @@ public class StreamingMultipleProgramsTestBase extends TestBaseUtils {
 
 	protected static final int DEFAULT_PARALLELISM = 4;
 
-	protected static ForkableFlinkMiniCluster cluster = null;
-	
-	// ------------------------------------------------------------------------
+	protected static ForkableFlinkMiniCluster cluster;
 	
-	public StreamingMultipleProgramsTestBase() {
-		TestStreamEnvironment clusterEnv = new TestStreamEnvironment(cluster, DEFAULT_PARALLELISM);
-		clusterEnv.setAsContext();
-	}
 
 	// ------------------------------------------------------------------------
 	//  Cluster setup & teardown
 	// ------------------------------------------------------------------------
 
 	@BeforeClass
-	public static void setup() throws Exception{
-		cluster = TestBaseUtils.startCluster(
-			1,
-			DEFAULT_PARALLELISM,
-			StreamingMode.STREAMING,
-			false,
-			false,
-			true);
+	public static void setup() throws Exception {
+		cluster = TestBaseUtils.startCluster(1, DEFAULT_PARALLELISM, StreamingMode.STREAMING, false, false, true);
+		TestStreamEnvironment.setAsContext(cluster, DEFAULT_PARALLELISM);
 	}
 
 	@AfterClass
 	public static void teardown() throws Exception {
+		TestStreamEnvironment.unsetAsContext();
 		stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
index 92f8301..ce3aa86 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
@@ -18,33 +18,27 @@
 
 package org.apache.flink.streaming.util;
 
-import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.test.util.AbstractTestBase;
 
-import org.junit.Assert;
 import org.junit.Test;
 
-public abstract class StreamingProgramTestBase extends AbstractTestBase {
-
-	private static final int DEFAULT_PARALLELISM = 4;
-
-	private TestStreamEnvironment env;
+import static org.junit.Assert.fail;
 
-	private JobExecutionResult latestExecutionResult;
-
-	private int parallelism = DEFAULT_PARALLELISM;
+public abstract class StreamingProgramTestBase extends AbstractTestBase {
 
+	protected static final int DEFAULT_PARALLELISM = 4;
 
+	private int parallelism;
+	
+	
 	public StreamingProgramTestBase() {
-		this(new Configuration());
+		super(new Configuration(), StreamingMode.STREAMING);
+		setParallelism(DEFAULT_PARALLELISM);
 	}
 
-	public StreamingProgramTestBase(Configuration config) {
-		super(config);
-		setTaskManagerNumSlots(parallelism);
-	}
-	
+
 	public void setParallelism(int parallelism) {
 		this.parallelism = parallelism;
 		setTaskManagerNumSlots(parallelism);
@@ -54,10 +48,6 @@ public abstract class StreamingProgramTestBase extends AbstractTestBase {
 		return parallelism;
 	}
 	
-	public JobExecutionResult getLatestExecutionResult() {
-		return this.latestExecutionResult;
-	}
-	
 
 	// --------------------------------------------------------------------------------------------
 	//  Methods to create the test program and for pre- and post- test work
@@ -74,8 +64,7 @@ public abstract class StreamingProgramTestBase extends AbstractTestBase {
 	// --------------------------------------------------------------------------------------------
 
 	@Test
-	public void testJobWithoutObjectReuse() throws Exception {
-		startCluster();
+	public void testJob() throws Exception {
 		try {
 			// pre-submit
 			try {
@@ -84,25 +73,26 @@ public abstract class StreamingProgramTestBase extends AbstractTestBase {
 			catch (Exception e) {
 				System.err.println(e.getMessage());
 				e.printStackTrace();
-				Assert.fail("Pre-submit work caused an error: " + e.getMessage());
+				fail("Pre-submit work caused an error: " + e.getMessage());
 			}
 
 			// prepare the test environment
-			env = new TestStreamEnvironment(this.executor, this.parallelism);
-			env.setAsContext();
+			startCluster();
+
+			TestStreamEnvironment.setAsContext(this.executor, getParallelism());
 
 			// call the test program
 			try {
 				testProgram();
-				this.latestExecutionResult = env.latestResult;
 			}
 			catch (Exception e) {
 				System.err.println(e.getMessage());
 				e.printStackTrace();
-				Assert.fail("Error while calling the test program: " + e.getMessage());
+				fail("Error while calling the test program: " + e.getMessage());
+			}
+			finally {
+				TestStreamEnvironment.unsetAsContext();
 			}
-
-			Assert.assertNotNull("The test program never triggered an execution.", this.latestExecutionResult);
 
 			// post-submit
 			try {
@@ -111,13 +101,11 @@ public abstract class StreamingProgramTestBase extends AbstractTestBase {
 			catch (Exception e) {
 				System.err.println(e.getMessage());
 				e.printStackTrace();
-				Assert.fail("Post-submit work caused an error: " + e.getMessage());
-			}
-		} finally {
-			if(env.clusterRunsSynchronous()) {
-				stopCluster();
+				fail("Post-submit work caused an error: " + e.getMessage());
 			}
 		}
+		finally {
+			stopCluster();
+		}
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
index 91082d8..8cd1e4a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
@@ -20,150 +20,56 @@ package org.apache.flink.streaming.util;
 
 import com.google.common.base.Preconditions;
 import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.client.SerializedJobExecutionResult;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
+/**
+ * A StreamExecutionEnvironment that executes its jobs on a test cluster.
+ */
 public class TestStreamEnvironment extends StreamExecutionEnvironment {
-	private static final String DEFAULT_JOBNAME = "TestStreamingJob";
-	private static final String CANNOT_EXECUTE_EMPTY_JOB = "Cannot execute empty job";
-
-	private long memorySize;
-	protected JobExecutionResult latestResult;
+	
+	/** The mini cluster in which this environment executes its jobs */
 	private ForkableFlinkMiniCluster executor;
-	private boolean internalExecutor;
-
-	public TestStreamEnvironment(int parallelism, long memorySize){
-		setParallelism(parallelism);
-		this.memorySize = memorySize;
-		internalExecutor = true;
-	}
+	
 
-	public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism){
+	public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism) {
 		this.executor = Preconditions.checkNotNull(executor);
-		setDefaultLocalParallelism(parallelism);
 		setParallelism(parallelism);
 	}
-
-	@Override
-	public JobExecutionResult execute() throws Exception {
-		return execute(DEFAULT_JOBNAME);
-	}
-
+	
 	@Override
 	public JobExecutionResult execute(String jobName) throws Exception {
-		JobExecutionResult result = execute(getStreamGraph().getJobGraph(jobName));
-		return result;
-	}
-	
-	public JobExecutionResult execute(JobGraph jobGraph) throws Exception {
-		if (internalExecutor) {
-			Configuration configuration = jobGraph.getJobConfiguration();
-
-			configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-					getParallelism());
-			configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize);
-
-			executor = new ForkableFlinkMiniCluster(configuration);
-			executor.start();
-		}
-		try {
-			sync = true;
-			latestResult = executor.submitJobAndWait(jobGraph, false);
-			return latestResult;
-		} catch (JobExecutionException e) {
-			if (e.getMessage().contains("GraphConversionException")) {
-				throw new Exception(CANNOT_EXECUTE_EMPTY_JOB, e);
-			} else {
-				throw e;
-			}
-		} finally {
-			transformations.clear();
-			if (internalExecutor){
-				executor.shutdown();
-			}
-		}
-	}
-
-	private ForkableFlinkMiniCluster cluster = null;
-	private Thread jobRunner = null;
-	private boolean sync = true;
-
-	public void start(final JobGraph jobGraph) throws Exception {
-		if(cluster != null) {
-			throw new IllegalStateException("The cluster is already running");
-		}
-
-		if (internalExecutor) {
-			Configuration configuration = jobGraph.getJobConfiguration();
-
-			configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-					getParallelism());
-			configuration.setLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, memorySize);
-
-			cluster = new ForkableFlinkMiniCluster(configuration);
-		} else {
-			cluster = executor;
-		}
-		try {
-			sync = false;
-
-			jobRunner = new Thread() {
-				public void run() {
-					try {
-						latestResult = cluster.submitJobAndWait(jobGraph, false);
-					} catch (JobExecutionException e) {
-						// TODO remove: hack to make ITCase succeed because .submitJobAndWait() throws exception on .stop() (see this.shutdown())
-						latestResult = new JobExecutionResult(null, 0, null);
-						e.printStackTrace();
-						//throw new RuntimeException(e);
-					} catch (Exception e) {
-						new RuntimeException(e);
-					}
-				}
-			};
-			jobRunner.start();
-		} catch(RuntimeException e) {
-			if (e.getCause().getMessage().contains("GraphConversionException")) {
-				throw new Exception(CANNOT_EXECUTE_EMPTY_JOB, e);
-			} else {
-				throw e;
-			}
-		}
-	}
-
-	public JobExecutionResult shutdown() throws InterruptedException {
-		if(!sync) {
-			cluster.stop();
-			cluster = null;
-
-			jobRunner.join();
-			jobRunner = null;
-
-			return latestResult;
-		}
-
-		throw new IllegalStateException("Cluster was not started via .start(...)");
-	}
-
-	public boolean clusterRunsSynchronous() {
-		return sync;
+		final JobGraph jobGraph = getStreamGraph().getJobGraph(jobName);
+		return executor.submitJobAndWait(jobGraph, false);
 	}
 
-	protected void setAsContext() {
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Sets the streaming context environment to a TestStreamEnvironment that runs its programs on
+	 * the given cluster with the given default parallelism.
+	 * 
+	 * @param cluster The test cluster to run the test program on.
+	 * @param parallelism The default parallelism for the test programs.
+	 */
+	public static void setAsContext(final ForkableFlinkMiniCluster cluster, final int parallelism) {
+		
 		StreamExecutionEnvironmentFactory factory = new StreamExecutionEnvironmentFactory() {
 			@Override
 			public StreamExecutionEnvironment createExecutionEnvironment() {
-				return TestStreamEnvironment.this;
+				return new TestStreamEnvironment(cluster, parallelism);
 			}
 		};
 
 		initializeContextEnvironment(factory);
 	}
 
+	/**
+	 * Resets the streaming context environment to null.
+	 */
+	public static void unsetAsContext() {
+		resetContextEnvironment();
+	} 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/82d62361/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
index fb0ab0a..37812c9 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
@@ -22,6 +22,7 @@ import org.apache.flink.streaming.examples.windowing.util.TopSpeedWindowingExamp
 import org.apache.flink.streaming.util.StreamingProgramTestBase;
 
 public class TopSpeedWindowingExampleITCase extends StreamingProgramTestBase {
+	
 	protected String textPath;
 	protected String resultPath;
 
@@ -40,6 +41,5 @@ public class TopSpeedWindowingExampleITCase extends StreamingProgramTestBase {
 	@Override
 	protected void testProgram() throws Exception {
 		TopSpeedWindowing.main(new String[]{textPath, resultPath});
-
 	}
 }


[9/9] flink git commit: [hotfix] [tests] Run IPv6HostnamesITCase with parallel data exchange and less verbose output

Posted by se...@apache.org.
[hotfix] [tests] Run IPv6HostnamesITCase with parallel data exchange and less verbose output


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

Branch: refs/heads/master
Commit: 846ad70642f755b39bd41f6f13b8f6aa5e98ac7d
Parents: 41717d1
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Oct 1 11:10:42 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 1 11:10:42 2015 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/846ad706/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
index e976c23..b4a5018 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
@@ -80,7 +80,9 @@ public class IPv6HostnamesITCase extends TestLogger {
 			flink.start();
 
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(addressString, flink.getLeaderRPCPort());
-
+			env.setParallelism(4);
+			env.getConfig().disableSysoutLogging();
+			
 			// get input data
 			DataSet<String> text = env.fromElements(WordCountData.TEXT.split("\n"));