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

[01/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Repository: flink
Updated Branches:
  refs/heads/master 42a3d8cfa -> 06f6ac5d3


http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
new file mode 100644
index 0000000..033e84f
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -0,0 +1,648 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.FoldFunction;
+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.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
+import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
+import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
+import org.apache.flink.streaming.api.functions.windowing.FoldWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+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.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
+import org.apache.flink.streaming.api.windowing.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.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+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 {@code WindowedStream} represents a data stream where elements are grouped by
+ * key, and for each key, the stream of elements is split into windows based on a
+ * {@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 WindowedStream} is purely and API construct, during runtime
+ * the {@code WindowedStream} will be collapsed together with the
+ * {@code KeyedStream} 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 WindowedStream<T, K, W extends Window> {
+
+	/** The keyed data stream that is windowed by this stream */
+	private final KeyedStream<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 WindowedStream(KeyedStream<T, K> input,
+			WindowAssigner<? super T, W> windowAssigner) {
+		this.input = input;
+		this.windowAssigner = windowAssigner;
+		this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
+	}
+
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	public WindowedStream<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 WindowedStream<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 SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "Reduce at " + callLocation;
+
+		SingleOutputStreamOperator<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;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
+		if (evictor != null) {
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					new ReduceWindowFunction<K, W, T>(function),
+					trigger,
+					evictor).enableSetProcessingTime(setProcessingTime);
+
+		} 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,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
+					new ReduceWindowFunction<K, W, T>(function),
+					trigger).enableSetProcessingTime(setProcessingTime);
+		}
+
+		return input.transform(opName, input.getType(), operator);
+	}
+
+	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
+				Utils.getCallLocationName(), true);
+
+		return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
+	}
+
+	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+		return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <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> SingleOutputStreamOperator<R, ?> apply(WindowFunction<T, R, K, W> function) {
+		TypeInformation<T> inType = input.getType();
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
+				function, WindowFunction.class, true, true, inType, null, false);
+
+		return apply(function, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <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.
+	 * @param resultType Type information for the result type of the window function
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> apply(WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "WindowApply at " + callLocation;
+
+		SingleOutputStreamOperator<R, ?> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		if (result != null) {
+			return result;
+		}
+
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+		KeySelector<T, K> keySel = input.getKeySelector();
+
+		WindowOperator<K, T, R, W> operator;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
+		if (evictor != null) {
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor).enableSetProcessingTime(setProcessingTime);
+
+		} else {
+			operator = new WindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger).enableSetProcessingTime(setProcessingTime);
+		}
+
+		return input.transform(opName, resultType, operator);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>
+	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
+	 *
+	 * @param preAggregator The reduce function that is used for 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> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, WindowFunction<T, R, K, W> function) {
+		TypeInformation<T> inType = input.getType();
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
+				function, WindowFunction.class, true, true, inType, null, false);
+
+		return apply(preAggregator, function, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>
+	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
+	 *
+	 * @param preAggregator The reduce function that is used for pre-aggregation
+	 * @param function The window function.
+	 * @param resultType Type information for the result type of the window function
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) {
+		//clean the closures
+		function = input.getExecutionEnvironment().clean(function);
+		preAggregator = input.getExecutionEnvironment().clean(preAggregator);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "WindowApply at " + callLocation;
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+		KeySelector<T, K> keySel = input.getKeySelector();
+
+		OneInputStreamOperator<T, R> operator;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
+		if (evictor != null) {
+			operator = new EvictingWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor).enableSetProcessingTime(setProcessingTime);
+
+		} else {
+			operator = new WindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					keySel,
+					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+					new PreAggregatingHeapWindowBuffer.Factory<>(preAggregator),
+					function,
+					trigger).enableSetProcessingTime(setProcessingTime);
+		}
+
+		return input.transform(opName, resultType, operator);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Aggregations on the keyed windows
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies an aggregation that sums every window of the data stream at the
+	 * given position.
+	 *
+	 * @param positionToSum The position in the tuple/array to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
+		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that sums every window of the pojo data stream at
+	 * the given field for every window.
+	 *
+	 * <p>
+	 * A field expression is either
+	 * the name of a public field or a getter method with parentheses of the
+	 * stream's underlying type. A dot can be used to drill down into objects,
+	 * as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(String field) {
+		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of every window
+	 * of the data stream at the given position.
+	 *
+	 * @param positionToMin The position to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
+		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of the pojo data
+	 * stream at the given field expression for every window.
+	 *
+	 * <p>
+	 * A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMinBy The position to minimize
+	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum element of the pojo
+	 * data stream by the given field expression for every window. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream DataStreams} underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @param first If True then in case of field equality the first object will be returned
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum value of every window of
+	 * the data stream at the given position.
+	 *
+	 * @param positionToMax The position to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
+		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum value of the pojo data
+	 * stream at the given field expression for every window. A field expression
+	 * is either the name of a public field or a getter method with parentheses
+	 * of the {@link DataStream DataStreams} underlying type. A dot can be used to drill
+	 * down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMaxBy The position to maximize by
+	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum element of the pojo
+	 * data stream by the given field expression for every window. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @param first If True then in case of field equality the first object will be returned
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	private SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregator) {
+		return reduce(aggregator);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private <R> SingleOutputStreamOperator<R, ?> createFastTimeOperatorIfValid(
+			Function function,
+			TypeInformation<R> resultType,
+			String functionName) {
+
+		if (windowAssigner instanceof SlidingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			SlidingTimeWindows timeWindows = (SlidingTimeWindows) 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(), 
+								input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+								input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+								windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+			else if (function instanceof WindowFunction) {
+				@SuppressWarnings("unchecked")
+				WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
+
+				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
+						wf, input.getKeySelector(),
+						input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+						input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+						windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+		} else if (windowAssigner instanceof TumblingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+			TumblingTimeWindows timeWindows = (TumblingTimeWindows) 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(),
+								input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+								input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+								windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+			else if (function instanceof WindowFunction) {
+				@SuppressWarnings("unchecked")
+				WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
+
+				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
+						wf, input.getKeySelector(),
+						input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+						input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+						windowLength, windowSlide);
+				return input.transform(opName, resultType, op);
+			}
+		}
+
+		return null;
+	}
+
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return input.getExecutionEnvironment();
+	}
+
+	public TypeInformation<T> getInputType() {
+		return input.getType();
+	}
+}


[10/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
new file mode 100644
index 0000000..715f5ee
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.apache.flink.util.NetUtils;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Properties;
+
+
+/**
+ * Flink Sink to produce data into a Kafka topic.
+ *
+ * Please note that this producer does not have any reliability guarantees.
+ *
+ * @param <IN> Type of the messages to write into Kafka.
+ */
+public class FlinkKafkaProducer<IN> extends RichSinkFunction<IN>  {
+
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class);
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Array with the partition ids of the given topicId
+	 * The size of this array is the number of partitions
+	 */
+	private final int[] partitions;
+
+	/**
+	 * User defined properties for the Producer
+	 */
+	private final Properties producerConfig;
+
+	/**
+	 * The name of the topic this producer is writing data to
+	 */
+	private final String topicId;
+
+	/**
+	 * (Serializable) SerializationSchema for turning objects used with Flink into
+	 * byte[] for Kafka.
+	 */
+	private final SerializationSchema<IN, byte[]> schema;
+
+	/**
+	 * User-provided partitioner for assigning an object to a Kafka partition.
+	 */
+	private final KafkaPartitioner partitioner;
+
+	/**
+	 * Flag indicating whether to accept failures (and log them), or to fail on failures
+	 */
+	private boolean logFailuresOnly;
+	
+	// -------------------------------- Runtime fields ------------------------------------------
+
+	/** KafkaProducer instance */
+	private transient KafkaProducer<byte[], byte[]> producer;
+
+	/** The callback than handles error propagation or logging callbacks */
+	private transient Callback callback;
+	
+	/** Errors encountered in the async producer are stored here */
+	private transient volatile Exception asyncException;
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param brokerList
+	 *			Comma separated addresses of the brokers
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema.
+	 */
+	public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema) {
+		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null);
+	}
+
+	/**
+	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
+	 * the topic.
+	 *
+	 * @param topicId
+	 * 			ID of the Kafka topic.
+	 * @param serializationSchema
+	 * 			User defined serialization schema.
+	 * @param producerConfig
+	 * 			Properties with the producer configuration.
+	 */
+	public FlinkKafkaProducer(String topicId, SerializationSchema<IN, byte[]> serializationSchema, Properties producerConfig) {
+		this(topicId, serializationSchema, producerConfig, null);
+	}
+
+	/**
+	 * The main constructor for creating a FlinkKafkaProducer.
+	 *
+	 * @param topicId The topic to write data to
+	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[]
+	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
+	 * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
+	 */
+	public FlinkKafkaProducer(String topicId, SerializationSchema<IN, byte[]> serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
+		Preconditions.checkNotNull(topicId, "TopicID not set");
+		Preconditions.checkNotNull(serializationSchema, "serializationSchema not set");
+		Preconditions.checkNotNull(producerConfig, "producerConfig not set");
+		ClosureCleaner.ensureSerializable(customPartitioner);
+		ClosureCleaner.ensureSerializable(serializationSchema);
+
+		this.topicId = topicId;
+		this.schema = serializationSchema;
+		this.producerConfig = producerConfig;
+
+		// set the producer configuration properties.
+
+		if(!producerConfig.contains(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
+			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
+		} else {
+			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
+		}
+
+		if(!producerConfig.contains(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
+			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
+		} else {
+			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
+		}
+
+
+		// create a local KafkaProducer to get the list of partitions.
+		// this will also ensure locally that all required ProducerConfig values are set.
+		try (KafkaProducer<Void, IN> getPartitionsProd = new KafkaProducer<>(this.producerConfig)) {
+			List<PartitionInfo> partitionsList = getPartitionsProd.partitionsFor(topicId);
+
+			this.partitions = new int[partitionsList.size()];
+			for (int i = 0; i < partitions.length; i++) {
+				partitions[i] = partitionsList.get(i).partition();
+			}
+			getPartitionsProd.close();
+		}
+
+		if (customPartitioner == null) {
+			this.partitioner = new FixedPartitioner();
+		} else {
+			this.partitioner = customPartitioner;
+		}
+	}
+
+	// ---------------------------------- Properties --------------------------
+
+	/**
+	 * Defines whether the producer should fail on errors, or only log them.
+	 * If this is set to true, then exceptions will be only logged, if set to false,
+	 * exceptions will be eventually thrown and cause the streaming program to 
+	 * fail (and enter recovery).
+	 * 
+	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
+	 */
+	public void setLogFailuresOnly(boolean logFailuresOnly) {
+		this.logFailuresOnly = logFailuresOnly;
+	}
+
+	// ----------------------------------- Utilities --------------------------
+	
+	/**
+	 * Initializes the connection to Kafka.
+	 */
+	@Override
+	public void open(Configuration configuration) {
+		producer = new org.apache.kafka.clients.producer.KafkaProducer<>(this.producerConfig);
+
+		RuntimeContext ctx = getRuntimeContext();
+		partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions);
+
+		LOG.info("Starting FlinkKafkaProducer ({}/{}) to produce into topic {}", 
+				ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), topicId);
+		
+		if (logFailuresOnly) {
+			callback = new Callback() {
+				
+				@Override
+				public void onCompletion(RecordMetadata metadata, Exception e) {
+					if (e != null) {
+						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
+					}
+				}
+			};
+		}
+		else {
+			callback = new Callback() {
+				@Override
+				public void onCompletion(RecordMetadata metadata, Exception exception) {
+					if (exception != null && asyncException == null) {
+						asyncException = exception;
+					}
+				}
+			};
+		}
+	}
+
+	/**
+	 * Called when new data arrives to the sink, and forwards it to Kafka.
+	 *
+	 * @param next
+	 * 		The incoming data
+	 */
+	@Override
+	public void invoke(IN next) throws Exception {
+		// propagate asynchronous errors
+		checkErroneous();
+		
+		byte[] serialized = schema.serialize(next);
+		ProducerRecord<byte[], byte[]> record = new ProducerRecord<byte[], byte[]>(topicId,
+				partitioner.partition(next, partitions.length),
+				null, serialized);
+		
+		producer.send(record, callback);
+	}
+
+
+	@Override
+	public void close() throws Exception {
+		if (producer != null) {
+			producer.close();
+		}
+		
+		// make sure we propagate pending errors
+		checkErroneous();
+	}
+
+
+	// ----------------------------------- Utilities --------------------------
+
+	private void checkErroneous() throws Exception {
+		Exception e = asyncException;
+		if (e != null) {
+			// prevent double throwing
+			asyncException = null;
+			throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e);
+		}
+	}
+	
+	public static Properties getPropertiesFromBrokerList(String brokerList) {
+		String[] elements = brokerList.split(",");
+		for(String broker: elements) {
+			NetUtils.getCorrectHostnamePort(broker);
+		}
+		Properties props = new Properties();
+		props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList);
+		return props;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
new file mode 100644
index 0000000..f856926
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka.api;
+
+
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+/**
+ * Sink that emits its inputs to a Kafka topic.
+ *
+ * The KafkaSink has been relocated to org.apache.flink.streaming.connectors.kafka.KafkaSink.
+ * This class will be removed in future releases of Flink.
+ */
+@Deprecated
+public class KafkaSink<IN> extends FlinkKafkaProducer<IN> {
+	public KafkaSink(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema) {
+		super(brokerList, topicId, serializationSchema);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
new file mode 100644
index 0000000..869c44f
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.api.persistent;
+
+import kafka.consumer.ConsumerConfig;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+
+/**
+ * Creates a Kafka consumer compatible with reading from Kafka 0.8.1+ consumers.
+ *
+ * This class is provided as a migration path from the old Flink kafka connectors to the new, updated implemntations.
+ *
+ * Please use FlinkKafkaConsumer081 and FlinkKafkaConsumer082.
+ *
+ * @param <T> The type of elements produced by this consumer.
+ */
+@Deprecated
+public class PersistentKafkaSource<T> extends FlinkKafkaConsumer<T> {
+
+	private static final long serialVersionUID = -8450689820627198228L;
+
+	/**
+	 * Creates a new Kafka 0.8.2.x streaming source consumer.
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
+	 * @param consumerConfig
+	 *           The consumer config used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public PersistentKafkaSource(String topic, DeserializationSchema<T> valueDeserializer, ConsumerConfig consumerConfig) {
+		super(topic, valueDeserializer, consumerConfig.props().props(), OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
new file mode 100644
index 0000000..4345926
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.kafka.common.TopicPartition;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A fetcher pulls data from Kafka, from a fix set of partitions.
+ * The fetcher supports "seeking" inside the partitions, i.e., moving to a different offset.
+ */
+public interface Fetcher {
+
+	/**
+	 * Set which partitions the fetcher should pull from.
+	 * 
+	 * @param partitions The list of partitions for a topic that the fetcher will pull from.
+	 */
+	void setPartitionsToRead(List<TopicPartition> partitions);
+
+	/**
+	 * Closes the fetcher. This will stop any operation in the
+	 * {@link #run(SourceFunction.SourceContext, DeserializationSchema, long[])} method and eventually
+	 * close underlying connections and release all resources.
+	 */
+	void close() throws IOException;
+
+	/**
+	 * Starts fetch data from Kafka and emitting it into the stream.
+	 * 
+	 * <p>To provide exactly once guarantees, the fetcher needs emit a record and update the update
+	 * of the last consumed offset in one atomic operation:</p>
+	 * <pre>{@code
+	 * 
+	 * while (running) {
+	 *     T next = ...
+	 *     long offset = ...
+	 *     int partition = ...
+	 *     synchronized (sourceContext.getCheckpointLock()) {
+	 *         sourceContext.collect(next);
+	 *         lastOffsets[partition] = offset;
+	 *     }
+	 * }
+	 * }</pre>
+	 * 
+	 * @param sourceContext The source context to emit elements to.
+	 * @param valueDeserializer The deserializer to decode the raw values with.
+	 * @param lastOffsets The array into which to store the offsets foe which elements are emitted. 
+	 * 
+	 * @param <T> The type of elements produced by the fetcher and emitted to the source context.
+	 */
+	<T> void run(SourceFunction.SourceContext<T> sourceContext, DeserializationSchema<T> valueDeserializer, 
+					long[] lastOffsets) throws Exception;
+	
+	/**
+	 * Set the next offset to read from for the given partition.
+	 * For example, if the partition <i>i</i> offset is set to <i>n</i>, the Fetcher's next result
+	 * will be the message with <i>offset=n</i>.
+	 * 
+	 * @param topicPartition The partition for which to seek the offset.
+	 * @param offsetToRead To offset to seek to.
+	 */
+	void seek(TopicPartition topicPartition, long offsetToRead);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
new file mode 100644
index 0000000..c4ba103
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
@@ -0,0 +1,622 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka.internals;
+
+import kafka.api.FetchRequestBuilder;
+import kafka.api.OffsetRequest;
+import kafka.api.PartitionOffsetRequestInfo;
+import kafka.common.ErrorMapping;
+import kafka.common.TopicAndPartition;
+import kafka.javaapi.FetchResponse;
+import kafka.javaapi.OffsetResponse;
+import kafka.javaapi.consumer.SimpleConsumer;
+import kafka.javaapi.message.ByteBufferMessageSet;
+import kafka.message.MessageAndOffset;
+
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * This fetcher uses Kafka's low-level API to pull data from a specific
+ * set of partitions and offsets for a certain topic.
+ * 
+ * <p>This code is in parts based on the tutorial code for the low-level Kafka consumer.</p>
+ */
+public class LegacyFetcher implements Fetcher {
+	
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer.class);
+
+	/** The topic from which this fetcher pulls data */
+	private final String topic;
+	
+	/** The properties that configure the Kafka connection */
+	private final Properties config;
+	
+	/** The task name, to give more readable names to the spawned threads */
+	private final String taskName;
+	
+	/** The first error that occurred in a connection thread */
+	private final AtomicReference<Throwable> error;
+
+	/** The partitions that the fetcher should read, with their starting offsets */
+	private Map<TopicPartition, Long> partitionsToRead;
+	
+	/** Reference the the thread that executed the run() method. */
+	private volatile Thread mainThread;
+	
+	/** Flag to shot the fetcher down */
+	private volatile boolean running = true;
+
+	public LegacyFetcher(String topic, Properties props, String taskName) {
+		this.config = checkNotNull(props, "The config properties cannot be null");
+		this.topic = checkNotNull(topic, "The topic cannot be null");
+		this.taskName = taskName;
+		this.error = new AtomicReference<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Fetcher methods
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public void setPartitionsToRead(List<TopicPartition> partitions) {
+		partitionsToRead = new HashMap<>(partitions.size());
+		for (TopicPartition tp: partitions) {
+			partitionsToRead.put(tp, FlinkKafkaConsumer.OFFSET_NOT_SET);
+		}
+	}
+
+	@Override
+	public void seek(TopicPartition topicPartition, long offsetToRead) {
+		if (partitionsToRead == null) {
+			throw new IllegalArgumentException("No partitions to read set");
+		}
+		if (!partitionsToRead.containsKey(topicPartition)) {
+			throw new IllegalArgumentException("Can not set offset on a partition (" + topicPartition
+					+ ") we are not going to read. Partitions to read " + partitionsToRead);
+		}
+		partitionsToRead.put(topicPartition, offsetToRead);
+	}
+	
+	@Override
+	public void close() {
+		// flag needs to be check by the run() method that creates the spawned threads
+		this.running = false;
+		
+		// all other cleanup is made by the run method itself
+	}
+
+	@Override
+	public <T> void run(SourceFunction.SourceContext<T> sourceContext, 
+						DeserializationSchema<T> valueDeserializer,
+						long[] lastOffsets) throws Exception {
+		
+		if (partitionsToRead == null || partitionsToRead.size() == 0) {
+			throw new IllegalArgumentException("No partitions set");
+		}
+		
+		// NOTE: This method is needs to always release all resources it acquires
+		
+		this.mainThread = Thread.currentThread();
+
+		LOG.info("Reading from partitions " + partitionsToRead + " using the legacy fetcher");
+		
+		// get lead broker for each partition
+		
+		// NOTE: The kafka client apparently locks itself in an infinite loop sometimes
+		// when it is interrupted, so we run it only in a separate thread.
+		// since it sometimes refuses to shut down, we resort to the admittedly harsh
+		// means of killing the thread after a timeout.
+		PartitionInfoFetcher infoFetcher = new PartitionInfoFetcher(topic, config);
+		infoFetcher.start();
+		
+		KillerWatchDog watchDog = new KillerWatchDog(infoFetcher, 60000);
+		watchDog.start();
+		
+		final List<PartitionInfo> allPartitionsInTopic = infoFetcher.getPartitions();
+		
+		// brokers to fetch partitions from.
+		int fetchPartitionsCount = 0;
+		Map<Node, List<FetchPartition>> fetchBrokers = new HashMap<>();
+		
+		for (PartitionInfo partitionInfo : allPartitionsInTopic) {
+			if (partitionInfo.leader() == null) {
+				throw new RuntimeException("Unable to consume partition " + partitionInfo.partition()
+						+ " from topic "+partitionInfo.topic()+" because it does not have a leader");
+			}
+			
+			for (Map.Entry<TopicPartition, Long> entry : partitionsToRead.entrySet()) {
+				final TopicPartition topicPartition = entry.getKey();
+				final long offset = entry.getValue();
+				
+				// check if that partition is for us
+				if (topicPartition.partition() == partitionInfo.partition()) {
+					List<FetchPartition> partitions = fetchBrokers.get(partitionInfo.leader());
+					if (partitions == null) {
+						partitions = new ArrayList<>();
+						fetchBrokers.put(partitionInfo.leader(), partitions);
+					}
+					
+					partitions.add(new FetchPartition(topicPartition.partition(), offset));
+					fetchPartitionsCount++;
+					
+				}
+				// else this partition is not for us
+			}
+		}
+		
+		if (partitionsToRead.size() != fetchPartitionsCount) {
+			throw new RuntimeException(partitionsToRead.size() + " partitions to read, but got only "
+					+ fetchPartitionsCount + " partition infos with lead brokers.");
+		}
+
+		// create SimpleConsumers for each broker
+		ArrayList<SimpleConsumerThread<?>> consumers = new ArrayList<>(fetchBrokers.size());
+		
+		for (Map.Entry<Node, List<FetchPartition>> brokerInfo : fetchBrokers.entrySet()) {
+			final Node broker = brokerInfo.getKey();
+			final List<FetchPartition> partitionsList = brokerInfo.getValue();
+			
+			FetchPartition[] partitions = partitionsList.toArray(new FetchPartition[partitionsList.size()]);
+
+			SimpleConsumerThread<T> thread = new SimpleConsumerThread<>(this, config, topic,
+					broker, partitions, sourceContext, valueDeserializer, lastOffsets);
+
+			thread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)",
+					taskName, broker.id(), broker.host(), broker.port()));
+			thread.setDaemon(true);
+			consumers.add(thread);
+		}
+		
+		// last check whether we should abort.
+		if (!running) {
+			return;
+		}
+		
+		// start all consumer threads
+		for (SimpleConsumerThread<?> t : consumers) {
+			LOG.info("Starting thread {}", t.getName());
+			t.start();
+		}
+		
+		// wait until all consumer threads are done, or until we are aborted, or until
+		// an error occurred in one of the fetcher threads
+		try {
+			boolean someConsumersRunning = true;
+			while (running && error.get() == null && someConsumersRunning) {
+				try {
+					// wait for the consumer threads. if an error occurs, we are interrupted
+					for (SimpleConsumerThread<?> t : consumers) {
+						t.join();
+					}
+	
+					// safety net
+					someConsumersRunning = false;
+					for (SimpleConsumerThread<?> t : consumers) {
+						someConsumersRunning |= t.isAlive();
+					}
+				}
+				catch (InterruptedException e) {
+					// ignore. we should notice what happened in the next loop check
+				}
+			}
+			
+			// make sure any asynchronous error is noticed
+			Throwable error = this.error.get();
+			if (error != null) {
+				throw new Exception(error.getMessage(), error);
+			}
+		}
+		finally {
+			// make sure that in any case (completion, abort, error), all spawned threads are stopped
+			for (SimpleConsumerThread<?> t : consumers) {
+				if (t.isAlive()) {
+					t.cancel();
+				}
+			}
+		}
+	}
+	
+	/**
+	 * Reports an error from a fetch thread. This will cause the main thread to see this error,
+	 * abort, and cancel all other fetch threads.
+	 * 
+	 * @param error The error to report.
+	 */
+	void onErrorInFetchThread(Throwable error) {
+		if (this.error.compareAndSet(null, error)) {
+			// we are the first to report an error
+			if (mainThread != null) {
+				mainThread.interrupt();
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Representation of a partition to fetch.
+	 */
+	private static class FetchPartition {
+		
+		/** ID of the partition within the topic (0 indexed, as given by Kafka) */
+		int partition;
+		
+		/** Offset pointing at the next element to read from that partition. */
+		long nextOffsetToRead;
+
+		FetchPartition(int partition, long nextOffsetToRead) {
+			this.partition = partition;
+			this.nextOffsetToRead = nextOffsetToRead;
+		}
+		
+		@Override
+		public String toString() {
+			return "FetchPartition {partition=" + partition + ", offset=" + nextOffsetToRead + '}';
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Per broker fetcher
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Each broker needs its separate connection. This thread implements the connection to
+	 * one broker. The connection can fetch multiple partitions from the broker.
+	 * 
+	 * @param <T> The data type fetched.
+	 */
+	private static class SimpleConsumerThread<T> extends Thread {
+		
+		private final SourceFunction.SourceContext<T> sourceContext;
+		private final DeserializationSchema<T> valueDeserializer;
+		private final long[] offsetsState;
+		
+		private final FetchPartition[] partitions;
+		
+		private final Node broker;
+		private final String topic;
+		private final Properties config;
+
+		private final LegacyFetcher owner;
+
+		private SimpleConsumer consumer;
+		
+		private volatile boolean running = true;
+
+
+		// exceptions are thrown locally
+		public SimpleConsumerThread(LegacyFetcher owner,
+									Properties config, String topic,
+									Node broker,
+									FetchPartition[] partitions,
+									SourceFunction.SourceContext<T> sourceContext,
+									DeserializationSchema<T> valueDeserializer,
+									long[] offsetsState) {
+			this.owner = owner;
+			this.config = config;
+			this.topic = topic;
+			this.broker = broker;
+			this.partitions = partitions;
+			this.sourceContext = checkNotNull(sourceContext);
+			this.valueDeserializer = checkNotNull(valueDeserializer);
+			this.offsetsState = checkNotNull(offsetsState);
+		}
+
+		@Override
+		public void run() {
+			try {
+				// set up the config values
+				final String clientId = "flink-kafka-consumer-legacy-" + broker.id();
+
+				// these are the actual configuration values of Kafka + their original default values.
+				final int soTimeout = Integer.valueOf(config.getProperty("socket.timeout.ms", "30000"));
+				final int bufferSize = Integer.valueOf(config.getProperty("socket.receive.buffer.bytes", "65536"));
+				final int fetchSize = Integer.valueOf(config.getProperty("fetch.message.max.bytes", "1048576"));
+				final int maxWait = Integer.valueOf(config.getProperty("fetch.wait.max.ms", "100"));
+				final int minBytes = Integer.valueOf(config.getProperty("fetch.min.bytes", "1"));
+				
+				// create the Kafka consumer that we actually use for fetching
+				consumer = new SimpleConsumer(broker.host(), broker.port(), soTimeout, bufferSize, clientId);
+
+				// make sure that all partitions have some offsets to start with
+				// those partitions that do not have an offset from a checkpoint need to get
+				// their start offset from ZooKeeper
+				{
+					List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<>();
+
+					for (FetchPartition fp : partitions) {
+						if (fp.nextOffsetToRead == FlinkKafkaConsumer.OFFSET_NOT_SET) {
+							// retrieve the offset from the consumer
+							partitionsToGetOffsetsFor.add(fp);
+						}
+					}
+					if (partitionsToGetOffsetsFor.size() > 0) {
+						getLastOffset(consumer, topic, partitionsToGetOffsetsFor, getInvalidOffsetBehavior(config));
+						LOG.info("No prior offsets found for some partitions in topic {}. Fetched the following start offsets {}",
+								topic, partitionsToGetOffsetsFor);
+					}
+				}
+				
+				// Now, the actual work starts :-)
+				int OffsetOutOfRangeCount = 0;
+				while (running) {
+					FetchRequestBuilder frb = new FetchRequestBuilder();
+					frb.clientId(clientId);
+					frb.maxWait(maxWait);
+					frb.minBytes(minBytes);
+					
+					for (FetchPartition fp : partitions) {
+						frb.addFetch(topic, fp.partition, fp.nextOffsetToRead, fetchSize);
+					}
+					kafka.api.FetchRequest fetchRequest = frb.build();
+					LOG.debug("Issuing fetch request {}", fetchRequest);
+
+					FetchResponse fetchResponse;
+					fetchResponse = consumer.fetch(fetchRequest);
+
+					if (fetchResponse.hasError()) {
+						String exception = "";
+						List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<>();
+						for (FetchPartition fp : partitions) {
+							short code = fetchResponse.errorCode(topic, fp.partition);
+
+							if(code == ErrorMapping.OffsetOutOfRangeCode()) {
+								// we were asked to read from an out-of-range-offset (maybe set wrong in Zookeeper)
+								// Kafka's high level consumer is resetting the offset according to 'auto.offset.reset'
+								partitionsToGetOffsetsFor.add(fp);
+							} else if(code != ErrorMapping.NoError()) {
+								exception += "\nException for partition " + fp.partition + ": " +
+										StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
+							}
+						}
+						if (partitionsToGetOffsetsFor.size() > 0) {
+							// safeguard against an infinite loop.
+							if(OffsetOutOfRangeCount++ > 0) {
+								throw new RuntimeException("Found invalid offsets more than once in partitions "+partitionsToGetOffsetsFor.toString()+" " +
+										"Exceptions: "+exception);
+							}
+							// get valid offsets for these partitions and try again.
+							LOG.warn("The following partitions had an invalid offset: {}", partitionsToGetOffsetsFor);
+							getLastOffset(consumer, topic, partitionsToGetOffsetsFor, getInvalidOffsetBehavior(config));
+							LOG.warn("The new partition offsets are {}", partitionsToGetOffsetsFor);
+							continue; // jump back to create a new fetch request. The offset has not been touched.
+						} else {
+							// all partitions failed on an error
+							throw new IOException("Error while fetching from broker: " + exception);
+						}
+					}
+
+					int messagesInFetch = 0;
+					for (FetchPartition fp : partitions) {
+						final ByteBufferMessageSet messageSet = fetchResponse.messageSet(topic, fp.partition);
+						final int partition = fp.partition;
+						
+						for (MessageAndOffset msg : messageSet) {
+							if (running) {
+								messagesInFetch++;
+								if (msg.offset() < fp.nextOffsetToRead) {
+									// we have seen this message already
+									LOG.info("Skipping message with offset " + msg.offset()
+											+ " because we have seen messages until " + fp.nextOffsetToRead
+											+ " from partition " + fp.partition + " already");
+									continue;
+								}
+								
+								ByteBuffer payload = msg.message().payload();
+								byte[] valueByte = new byte[payload.remaining()];
+								payload.get(valueByte);
+								
+								final T value = valueDeserializer.deserialize(valueByte);
+								final long offset = msg.offset();
+										
+								synchronized (sourceContext.getCheckpointLock()) {
+									sourceContext.collect(value);
+									offsetsState[partition] = offset;
+								}
+								
+								// advance offset for the next request
+								fp.nextOffsetToRead = offset + 1;
+							}
+							else {
+								// no longer running
+								return;
+							}
+						}
+					}
+					LOG.debug("This fetch contained {} messages", messagesInFetch);
+				}
+			}
+			catch (Throwable t) {
+				// report to the main thread
+				owner.onErrorInFetchThread(t);
+			}
+			finally {
+				// end of run loop. close connection to consumer
+				if (consumer != null) {
+					// closing the consumer should not fail the program
+					try {
+						consumer.close();
+					}
+					catch (Throwable t) {
+						LOG.error("Error while closing the Kafka simple consumer", t);
+					}
+				}
+			}
+		}
+
+		/**
+		 * Cancels this fetch thread. The thread will release all resources and terminate.
+		 */
+		public void cancel() {
+			this.running = false;
+			
+			// interrupt whatever the consumer is doing
+			if (consumer != null) {
+				consumer.close();
+			}
+			
+			this.interrupt();
+		}
+
+		/**
+		 * Request latest offsets for a set of partitions, via a Kafka consumer.
+		 *
+		 * @param consumer The consumer connected to lead broker
+		 * @param topic The topic name
+		 * @param partitions The list of partitions we need offsets for
+		 * @param whichTime The type of time we are requesting. -1 and -2 are special constants (See OffsetRequest)
+		 */
+		private static void getLastOffset(SimpleConsumer consumer, String topic, List<FetchPartition> partitions, long whichTime) {
+
+			Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<>();
+			for (FetchPartition fp: partitions) {
+				TopicAndPartition topicAndPartition = new TopicAndPartition(topic, fp.partition);
+				requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(whichTime, 1));
+			}
+
+			kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
+			OffsetResponse response = consumer.getOffsetsBefore(request);
+
+			if (response.hasError()) {
+				String exception = "";
+				for (FetchPartition fp: partitions) {
+					short code;
+					if ( (code=response.errorCode(topic, fp.partition)) != ErrorMapping.NoError()) {
+						exception += "\nException for partition "+fp.partition+": "+ StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
+					}
+				}
+				throw new RuntimeException("Unable to get last offset for topic " + topic + " and partitions " + partitions
+						+ ". " + exception);
+			}
+
+			for (FetchPartition fp: partitions) {
+				// the resulting offset is the next offset we are going to read
+				// for not-yet-consumed partitions, it is 0.
+				fp.nextOffsetToRead = response.offsets(topic, fp.partition)[0];
+			}
+		}
+
+		private static long getInvalidOffsetBehavior(Properties config) {
+			long timeType;
+			if (config.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest").equals("latest")) {
+				timeType = OffsetRequest.LatestTime();
+			} else {
+				timeType = OffsetRequest.EarliestTime();
+			}
+			return timeType;
+		}
+	}
+	
+	private static class PartitionInfoFetcher extends Thread {
+
+		private final String topic;
+		private final Properties properties;
+		
+		private volatile List<PartitionInfo> result;
+		private volatile Throwable error;
+
+		
+		PartitionInfoFetcher(String topic, Properties properties) {
+			this.topic = topic;
+			this.properties = properties;
+		}
+
+		@Override
+		public void run() {
+			try {
+				result = FlinkKafkaConsumer.getPartitionsForTopic(topic, properties);
+			}
+			catch (Throwable t) {
+				this.error = t;
+			}
+		}
+		
+		public List<PartitionInfo> getPartitions() throws Exception {
+			try {
+				this.join();
+			}
+			catch (InterruptedException e) {
+				throw new Exception("Partition fetching was cancelled before completion");
+			}
+			
+			if (error != null) {
+				throw new Exception("Failed to fetch partitions for topic " + topic, error);
+			}
+			if (result != null) {
+				return result;
+			}
+			throw new Exception("Partition fetching failed");
+		}
+	}
+
+	private static class KillerWatchDog extends Thread {
+		
+		private final Thread toKill;
+		private final long timeout;
+
+		private KillerWatchDog(Thread toKill, long timeout) {
+			super("KillerWatchDog");
+			setDaemon(true);
+			
+			this.toKill = toKill;
+			this.timeout = timeout;
+		}
+
+		@SuppressWarnings("deprecation")
+		@Override
+		public void run() {
+			final long deadline = System.currentTimeMillis() + timeout;
+			long now;
+			
+			while (toKill.isAlive() && (now = System.currentTimeMillis()) < deadline) {
+				try {
+					toKill.join(deadline - now);
+				}
+				catch (InterruptedException e) {
+					// ignore here, our job is important!
+				}
+			}
+			
+			// this is harsh, but this watchdog is a last resort
+			if (toKill.isAlive()) {
+				toKill.stop();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
new file mode 100644
index 0000000..2a82561
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+
+import org.apache.kafka.common.TopicPartition;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The offset handler is responsible for locating the initial partition offsets 
+ * where the source should start reading, as well as committing offsets from completed
+ * checkpoints.
+ */
+public interface OffsetHandler {
+
+	/**
+	 * Commits the given offset for the partitions. May commit the offsets to the Kafka broker,
+	 * or to ZooKeeper, based on its configured behavior.
+	 *
+	 * @param offsetsToCommit The offset to commit, per partition.
+	 */
+	void commit(Map<TopicPartition, Long> offsetsToCommit) throws Exception;
+
+	/**
+	 * Positions the given fetcher to the initial read offsets where the stream consumption
+	 * will start from.
+	 * 
+	 * @param partitions The partitions for which to seeks the fetcher to the beginning.
+	 * @param fetcher The fetcher that will pull data from Kafka and must be positioned.
+	 */
+	void seekFetcherToInitialOffsets(List<TopicPartition> partitions, Fetcher fetcher) throws Exception;
+
+	/**
+	 * Closes the offset handler, releasing all resources.
+	 * 
+	 * @throws IOException Thrown, if the closing fails.
+	 */
+	void close() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
new file mode 100644
index 0000000..a38c3bd
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import kafka.producer.Partitioner;
+import kafka.utils.VerifiableProperties;
+
+/**
+ * Hacky wrapper to send an object instance through a Properties - map.
+ *
+ * This works as follows:
+ * The recommended way of creating a KafkaSink is specifying a classname for the partitioner.
+ *
+ * Otherwise (if the user gave a (serializable) class instance), we give Kafka the PartitionerWrapper class of Flink.
+ * This is set in the key-value (java.util.Properties) map.
+ * In addition to that, we use the Properties.put(Object, Object) to store the instance of the (serializable).
+ * This is a hack because the put() method is called on the underlying Hashmap.
+ *
+ * This PartitionerWrapper is called with the Properties. From there, we extract the wrapped Partitioner instance.
+ *
+ * The serializable PartitionerWrapper is serialized into the Properties Hashmap and also deserialized from there.
+ */
+public class PartitionerWrapper implements Partitioner {
+	public final static String SERIALIZED_WRAPPER_NAME = "flink.kafka.wrapper.serialized";
+
+	private Partitioner wrapped;
+	public PartitionerWrapper(VerifiableProperties properties) {
+		wrapped = (Partitioner) properties.props().get(SERIALIZED_WRAPPER_NAME);
+	}
+
+	@Override
+	public int partition(Object value, int numberOfPartitions) {
+		return wrapped.partition(value, numberOfPartitions);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
new file mode 100644
index 0000000..001b6cb
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka.internals;
+
+import org.I0Itec.zkclient.serialize.ZkSerializer;
+
+import java.nio.charset.Charset;
+
+/**
+ * Simple ZooKeeper serializer for Strings.
+ */
+public class ZooKeeperStringSerializer implements ZkSerializer {
+
+	private static final Charset CHARSET = Charset.forName("UTF-8");
+	
+	@Override
+	public byte[] serialize(Object data) {
+		if (data instanceof String) {
+			return ((String) data).getBytes(CHARSET);
+		}
+		else {
+			throw new IllegalArgumentException("ZooKeeperStringSerializer can only serialize strings.");
+		}
+	}
+
+	@Override
+	public Object deserialize(byte[] bytes) {
+		if (bytes == null) {
+			return null;
+		}
+		else {
+			return new String(bytes, CHARSET);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
new file mode 100644
index 0000000..42a5951
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka.internals;
+
+import kafka.common.TopicAndPartition;
+import kafka.utils.ZKGroupTopicDirs;
+import kafka.utils.ZkUtils;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.zookeeper.data.Stat;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.Option;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class ZookeeperOffsetHandler implements OffsetHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class);
+	
+	private static final long OFFSET_NOT_SET = FlinkKafkaConsumer.OFFSET_NOT_SET;
+	
+	
+	private final ZkClient zkClient;
+	
+	private final String groupId;
+
+	
+	public ZookeeperOffsetHandler(Properties props) {
+		this.groupId = props.getProperty(ConsumerConfig.GROUP_ID_CONFIG);
+		
+		if (this.groupId == null) {
+			throw new IllegalArgumentException("Required property '"
+					+ ConsumerConfig.GROUP_ID_CONFIG + "' has not been set");
+		}
+		
+		String zkConnect = props.getProperty("zookeeper.connect");
+		if (zkConnect == null) {
+			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set");
+		}
+		
+		zkClient = new ZkClient(zkConnect,
+				Integer.valueOf(props.getProperty("zookeeper.session.timeout.ms", "6000")),
+				Integer.valueOf(props.getProperty("zookeeper.connection.timeout.ms", "6000")),
+				new ZooKeeperStringSerializer());
+	}
+
+
+	@Override
+	public void commit(Map<TopicPartition, Long> offsetsToCommit) {
+		for (Map.Entry<TopicPartition, Long> entry : offsetsToCommit.entrySet()) {
+			TopicPartition tp = entry.getKey();
+			long offset = entry.getValue();
+			
+			if (offset >= 0) {
+				setOffsetInZooKeeper(zkClient, groupId, tp.topic(), tp.partition(), offset);
+			}
+		}
+	}
+
+	@Override
+	public void seekFetcherToInitialOffsets(List<TopicPartition> partitions, Fetcher fetcher) {
+		for (TopicPartition tp : partitions) {
+			long offset = getOffsetFromZooKeeper(zkClient, groupId, tp.topic(), tp.partition());
+
+			if (offset != OFFSET_NOT_SET) {
+				LOG.info("Offset for partition {} was set to {} in ZooKeeper. Seeking fetcher to that position.",
+						tp.partition(), offset);
+
+				// the offset in Zookeeper was the last read offset, seek is accepting the next-to-read-offset.
+				fetcher.seek(tp, offset + 1);
+			}
+		}
+	}
+
+	@Override
+	public void close() throws IOException {
+		zkClient.close();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Communication with Zookeeper
+	// ------------------------------------------------------------------------
+	
+	public static void setOffsetInZooKeeper(ZkClient zkClient, String groupId, String topic, int partition, long offset) {
+		TopicAndPartition tap = new TopicAndPartition(topic, partition);
+		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
+		ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir() + "/" + tap.partition(), Long.toString(offset));
+	}
+
+	public static long getOffsetFromZooKeeper(ZkClient zkClient, String groupId, String topic, int partition) {
+		TopicAndPartition tap = new TopicAndPartition(topic, partition);
+		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
+
+		scala.Tuple2<Option<String>, Stat> data = ZkUtils.readDataMaybeNull(zkClient,
+				topicDirs.consumerOffsetDir() + "/" + tap.partition());
+
+		if (data._1().isEmpty()) {
+			return OFFSET_NOT_SET;
+		} else {
+			return Long.valueOf(data._1().get());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
new file mode 100644
index 0000000..346a7d5
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka.partitioner;
+
+import java.io.Serializable;
+
+/**
+ * A partitioner ensuring that each internal Flink partition ends up in one Kafka partition.
+ *
+ * Note, one Kafka partition can contain multiple Flink partitions.
+ *
+ * Cases:
+ * 	# More Flink partitions than kafka partitions
+ * <pre>
+ * 		Flink Sinks:		Kafka Partitions
+ * 			1	---------------->	1
+ * 			2   --------------/
+ * 			3   -------------/
+ * 			4	------------/
+ * </pre>
+ * 	--> Some (or all) kafka partitions contain the output of more than one flink partition
+ *
+ *# Fewer Flink partitions than Kafka
+ * <pre>
+ * 		Flink Sinks:		Kafka Partitions
+ * 			1	---------------->	1
+ * 			2	---------------->	2
+ * 									3
+ * 									4
+ * 									5
+ * </pre>
+ *
+ *  --> Not all Kafka partitions contain data
+ *  To avoid such an unbalanced partitioning, use a round-robin kafka partitioner. (note that this will
+ *  cause a lot of network connections between all the Flink instances and all the Kafka brokers
+ *
+ *
+ */
+public class FixedPartitioner extends KafkaPartitioner implements Serializable {
+	private static final long serialVersionUID = 1627268846962918126L;
+
+	int targetPartition = -1;
+
+	@Override
+	public void open(int parallelInstanceId, int parallelInstances, int[] partitions) {
+		int p = 0;
+		for(int i = 0; i < parallelInstances; i++) {
+			if(i == parallelInstanceId) {
+				targetPartition = partitions[p];
+				return;
+			}
+			if(++p == partitions.length) {
+				p = 0;
+			}
+		}
+	}
+
+	@Override
+	public int partition(Object element, int numPartitions) {
+		if(targetPartition == -1) {
+			throw new RuntimeException("The partitioner has not been initialized properly");
+		}
+		return targetPartition;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
new file mode 100644
index 0000000..55519f0
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.kafka.partitioner;
+
+
+import kafka.producer.Partitioner;
+
+import java.io.Serializable;
+
+/**
+ * Extended Kafka Partitioner.
+ * It contains a open() method which is called on each parallel instance.
+ * Partitioners have to be serializable!
+ */
+public abstract class KafkaPartitioner implements Partitioner, Serializable {
+
+	private static final long serialVersionUID = -1974260817778593473L;
+
+	/**
+	 * Initializer for the Partitioner.
+	 * @param parallelInstanceId 0-indexed id of the parallel instance in Flink
+	 * @param parallelInstances the total number of parallel instances
+	 * @param partitions an array describing the partition IDs of the available Kafka partitions.
+	 */
+	public void open(int parallelInstanceId, int parallelInstances, int[] partitions) {
+		// overwrite this method if needed.
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
new file mode 100644
index 0000000..3d392aa
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.kafka.common.TopicPartition;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests that the partition assignment is deterministic and stable.
+ */
+public class KafkaConsumerPartitionAssignmentTest {
+
+	@Test
+	public void testPartitionsEqualConsumers() {
+		try {
+			int[] partitions = {4, 52, 17, 1};
+			
+			for (int i = 0; i < partitions.length; i++) {
+				List<TopicPartition> parts = FlinkKafkaConsumer.assignPartitions(
+						partitions, "test-topic", partitions.length, i);
+				
+				assertNotNull(parts);
+				assertEquals(1, parts.size());
+				assertTrue(contains(partitions, parts.get(0).partition()));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testMultiplePartitionsPerConsumers() {
+		try {
+			final int[] partitions = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14};
+
+			final Set<Integer> allPartitions = new HashSet<>();
+			for (int i : partitions) {
+				allPartitions.add(i);
+			}
+			
+			final int numConsumers = 3;
+			final int minPartitionsPerConsumer = partitions.length / numConsumers;
+			final int maxPartitionsPerConsumer = partitions.length / numConsumers + 1;
+			
+			for (int i = 0; i < numConsumers; i++) {
+				List<TopicPartition> parts = FlinkKafkaConsumer.assignPartitions(
+						partitions, "test-topic", numConsumers, i);
+
+				assertNotNull(parts);
+				assertTrue(parts.size() >= minPartitionsPerConsumer);
+				assertTrue(parts.size() <= maxPartitionsPerConsumer);
+
+				for (TopicPartition p : parts) {
+					// check that the element was actually contained
+					assertTrue(allPartitions.remove(p.partition()));
+				}
+			}
+
+			// all partitions must have been assigned
+			assertTrue(allPartitions.isEmpty());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testPartitionsFewerThanConsumers() {
+		try {
+			final int[] partitions = {4, 52, 17, 1};
+
+			final Set<Integer> allPartitions = new HashSet<>();
+			for (int i : partitions) {
+				allPartitions.add(i);
+			}
+
+			final int numConsumers = 2 * partitions.length + 3;
+			
+			for (int i = 0; i < numConsumers; i++) {
+				List<TopicPartition> parts = FlinkKafkaConsumer.assignPartitions(
+						partitions, "test-topic", numConsumers, i);
+
+				assertNotNull(parts);
+				assertTrue(parts.size() <= 1);
+				
+				for (TopicPartition p : parts) {
+					// check that the element was actually contained
+					assertTrue(allPartitions.remove(p.partition()));
+				}
+			}
+
+			// all partitions must have been assigned
+			assertTrue(allPartitions.isEmpty());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testAssignEmptyPartitions() {
+		try {
+			List<TopicPartition> parts1 = FlinkKafkaConsumer.assignPartitions(new int[0], "test-topic", 4, 2);
+			assertNotNull(parts1);
+			assertTrue(parts1.isEmpty());
+
+			List<TopicPartition> parts2 = FlinkKafkaConsumer.assignPartitions(new int[0], "test-topic", 1, 0);
+			assertNotNull(parts2);
+			assertTrue(parts2.isEmpty());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testGrowingPartitionsRemainsStable() {
+		try {
+			final int[] newPartitions = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14};
+			final int[] initialPartitions = Arrays.copyOfRange(newPartitions, 0, 7);
+
+			final Set<Integer> allNewPartitions = new HashSet<>();
+			final Set<Integer> allInitialPartitions = new HashSet<>();
+			for (int i : newPartitions) {
+				allNewPartitions.add(i);
+			}
+			for (int i : initialPartitions) {
+				allInitialPartitions.add(i);
+			}
+
+			final int numConsumers = 3;
+			final int minInitialPartitionsPerConsumer = initialPartitions.length / numConsumers;
+			final int maxInitialPartitionsPerConsumer = initialPartitions.length / numConsumers + 1;
+			final int minNewPartitionsPerConsumer = newPartitions.length / numConsumers;
+			final int maxNewPartitionsPerConsumer = newPartitions.length / numConsumers + 1;
+			
+			List<TopicPartition> parts1 = FlinkKafkaConsumer.assignPartitions(
+					initialPartitions, "test-topic", numConsumers, 0);
+			List<TopicPartition> parts2 = FlinkKafkaConsumer.assignPartitions(
+					initialPartitions, "test-topic", numConsumers, 1);
+			List<TopicPartition> parts3 = FlinkKafkaConsumer.assignPartitions(
+					initialPartitions, "test-topic", numConsumers, 2);
+
+			assertNotNull(parts1);
+			assertNotNull(parts2);
+			assertNotNull(parts3);
+			
+			assertTrue(parts1.size() >= minInitialPartitionsPerConsumer);
+			assertTrue(parts1.size() <= maxInitialPartitionsPerConsumer);
+			assertTrue(parts2.size() >= minInitialPartitionsPerConsumer);
+			assertTrue(parts2.size() <= maxInitialPartitionsPerConsumer);
+			assertTrue(parts3.size() >= minInitialPartitionsPerConsumer);
+			assertTrue(parts3.size() <= maxInitialPartitionsPerConsumer);
+
+			for (TopicPartition p : parts1) {
+				// check that the element was actually contained
+				assertTrue(allInitialPartitions.remove(p.partition()));
+			}
+			for (TopicPartition p : parts2) {
+				// check that the element was actually contained
+				assertTrue(allInitialPartitions.remove(p.partition()));
+			}
+			for (TopicPartition p : parts3) {
+				// check that the element was actually contained
+				assertTrue(allInitialPartitions.remove(p.partition()));
+			}
+			
+			// all partitions must have been assigned
+			assertTrue(allInitialPartitions.isEmpty());
+			
+			// grow the set of partitions and distribute anew
+			
+			List<TopicPartition> parts1new = FlinkKafkaConsumer.assignPartitions(
+					newPartitions, "test-topic", numConsumers, 0);
+			List<TopicPartition> parts2new = FlinkKafkaConsumer.assignPartitions(
+					newPartitions, "test-topic", numConsumers, 1);
+			List<TopicPartition> parts3new = FlinkKafkaConsumer.assignPartitions(
+					newPartitions, "test-topic", numConsumers, 2);
+
+			// new partitions must include all old partitions
+			
+			assertTrue(parts1new.size() > parts1.size());
+			assertTrue(parts2new.size() > parts2.size());
+			assertTrue(parts3new.size() > parts3.size());
+			
+			assertTrue(parts1new.containsAll(parts1));
+			assertTrue(parts2new.containsAll(parts2));
+			assertTrue(parts3new.containsAll(parts3));
+
+			assertTrue(parts1new.size() >= minNewPartitionsPerConsumer);
+			assertTrue(parts1new.size() <= maxNewPartitionsPerConsumer);
+			assertTrue(parts2new.size() >= minNewPartitionsPerConsumer);
+			assertTrue(parts2new.size() <= maxNewPartitionsPerConsumer);
+			assertTrue(parts3new.size() >= minNewPartitionsPerConsumer);
+			assertTrue(parts3new.size() <= maxNewPartitionsPerConsumer);
+
+			for (TopicPartition p : parts1new) {
+				// check that the element was actually contained
+				assertTrue(allNewPartitions.remove(p.partition()));
+			}
+			for (TopicPartition p : parts2new) {
+				// check that the element was actually contained
+				assertTrue(allNewPartitions.remove(p.partition()));
+			}
+			for (TopicPartition p : parts3new) {
+				// check that the element was actually contained
+				assertTrue(allNewPartitions.remove(p.partition()));
+			}
+
+			// all partitions must have been assigned
+			assertTrue(allNewPartitions.isEmpty());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private static boolean contains(int[] array, int value) {
+		for (int i : array) {
+			if (i == value) {
+				return true;
+			}
+		}
+		return false;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
new file mode 100644
index 0000000..e35fcfb
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.commons.collections.map.LinkedMap;
+
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
+import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.Properties;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+public class KafkaConsumerTest {
+
+	@Test
+	public void testValidateZooKeeperConfig() {
+		try {
+			// empty
+			Properties emptyProperties = new Properties();
+			try {
+				FlinkKafkaConsumer.validateZooKeeperConfig(emptyProperties);
+				fail("should fail with an exception");
+			}
+			catch (IllegalArgumentException e) {
+				// expected
+			}
+
+			// no connect string (only group string)
+			Properties noConnect = new Properties();
+			noConnect.put(ConsumerConfig.GROUP_ID_CONFIG, "flink-test-group");
+			try {
+				FlinkKafkaConsumer.validateZooKeeperConfig(noConnect);
+				fail("should fail with an exception");
+			}
+			catch (IllegalArgumentException e) {
+				// expected
+			}
+
+			// no group string (only connect string)
+			Properties noGroup = new Properties();
+			noGroup.put("zookeeper.connect", "localhost:47574");
+			try {
+				FlinkKafkaConsumer.validateZooKeeperConfig(noGroup);
+				fail("should fail with an exception");
+			}
+			catch (IllegalArgumentException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSnapshot() {
+		try {
+			Field offsetsField = FlinkKafkaConsumer.class.getDeclaredField("lastOffsets");
+			Field runningField = FlinkKafkaConsumer.class.getDeclaredField("running");
+			Field mapField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints");
+			
+			offsetsField.setAccessible(true);
+			runningField.setAccessible(true);
+			mapField.setAccessible(true);
+
+			FlinkKafkaConsumer<?> consumer = mock(FlinkKafkaConsumer.class);
+			when(consumer.snapshotState(anyLong(), anyLong())).thenCallRealMethod();
+			
+			long[] testOffsets = new long[] { 43, 6146, 133, 16, 162, 616 };
+			LinkedMap map = new LinkedMap();
+			
+			offsetsField.set(consumer, testOffsets);
+			runningField.set(consumer, true);
+			mapField.set(consumer, map);
+			
+			assertTrue(map.isEmpty());
+
+			// make multiple checkpoints
+			for (long checkpointId = 10L; checkpointId <= 2000L; checkpointId += 9L) {
+				long[] checkpoint = consumer.snapshotState(checkpointId, 47 * checkpointId);
+				assertArrayEquals(testOffsets, checkpoint);
+				
+				// change the offsets, make sure the snapshot did not change
+				long[] checkpointCopy = Arrays.copyOf(checkpoint, checkpoint.length);
+				
+				for (int i = 0; i < testOffsets.length; i++) {
+					testOffsets[i] += 1L;
+				}
+				
+				assertArrayEquals(checkpointCopy, checkpoint);
+				
+				assertTrue(map.size() > 0);
+				assertTrue(map.size() <= FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	@Ignore("Kafka consumer internally makes an infinite loop")
+	public void testCreateSourceWithoutCluster() {
+		try {
+			Properties props = new Properties();
+			props.setProperty("zookeeper.connect", "localhost:56794");
+			props.setProperty("bootstrap.servers", "localhost:11111, localhost:22222");
+			props.setProperty("group.id", "non-existent-group");
+
+			new FlinkKafkaConsumer<>("no op topic", new JavaDefaultStringSchema(), props,
+					FlinkKafkaConsumer.OffsetStore.FLINK_ZOOKEEPER,
+					FlinkKafkaConsumer.FetcherType.LEGACY_LOW_LEVEL);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[51/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
[FLINK-2877] Move Streaming API out of Staging package


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

Branch: refs/heads/master
Commit: 06f6ac5d3edbc8dd20baf6e1b30900c189a68876
Parents: 42a3d8c
Author: Rufus Refactor <ru...@refactor.com>
Authored: Tue Oct 20 17:04:21 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 21 11:01:43 2015 +0200

----------------------------------------------------------------------
 docs/apis/best_practices.md                     |    2 +-
 docs/apis/streaming_guide.md                    |    2 +-
 docs/internals/stream_checkpointing.md          |    2 +-
 flink-contrib/flink-storm-examples/pom.xml      |    2 +-
 flink-contrib/flink-storm/pom.xml               |    2 +-
 flink-contrib/flink-streaming-contrib/pom.xml   |    2 +-
 flink-dist/pom.xml                              |    2 +-
 flink-java8/pom.xml                             |    2 +-
 .../main/resources/archetype-resources/pom.xml  |    6 +-
 .../main/resources/archetype-resources/pom.xml  |    4 +-
 flink-staging/flink-fs-tests/pom.xml            |    2 +-
 flink-staging/flink-hbase/pom.xml               |    2 +-
 .../flink-connector-elasticsearch/pom.xml       |  106 --
 .../elasticsearch/ElasticsearchSink.java        |  315 ----
 .../elasticsearch/IndexRequestBuilder.java      |   66 -
 .../examples/ElasticsearchExample.java          |   81 -
 .../elasticsearch/ElasticsearchSinkITCase.java  |  205 ---
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-filesystem/pom.xml          |  112 --
 .../flink/streaming/connectors/fs/Bucketer.java |   52 -
 .../flink/streaming/connectors/fs/Clock.java    |   32 -
 .../connectors/fs/DateTimeBucketer.java         |  124 --
 .../connectors/fs/NonRollingBucketer.java       |   43 -
 .../streaming/connectors/fs/RollingSink.java    |  900 -----------
 .../connectors/fs/SequenceFileWriter.java       |  160 --
 .../streaming/connectors/fs/StringWriter.java   |  103 --
 .../streaming/connectors/fs/SystemClock.java    |   28 -
 .../flink/streaming/connectors/fs/Writer.java   |   64 -
 .../src/main/resources/log4j.properties         |   27 -
 .../fs/RollingSinkFaultTolerance2ITCase.java    |  289 ----
 .../fs/RollingSinkFaultToleranceITCase.java     |  284 ----
 .../connectors/fs/RollingSinkITCase.java        |  506 ------
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-flume/pom.xml               |  174 ---
 .../streaming/connectors/flume/FlumeSink.java   |  141 --
 .../streaming/connectors/flume/FlumeSource.java |  149 --
 .../connectors/flume/FlumeTopology.java         |   49 -
 .../flink-connector-kafka/pom.xml               |  130 --
 .../connectors/kafka/FlinkKafkaConsumer.java    |  689 ---------
 .../connectors/kafka/FlinkKafkaConsumer081.java |   57 -
 .../connectors/kafka/FlinkKafkaConsumer082.java |   51 -
 .../connectors/kafka/FlinkKafkaProducer.java    |  288 ----
 .../connectors/kafka/api/KafkaSink.java         |   34 -
 .../api/persistent/PersistentKafkaSource.java   |   52 -
 .../connectors/kafka/internals/Fetcher.java     |   83 -
 .../kafka/internals/LegacyFetcher.java          |  622 --------
 .../kafka/internals/OffsetHandler.java          |   58 -
 .../kafka/internals/PartitionerWrapper.java     |   49 -
 .../internals/ZooKeeperStringSerializer.java    |   51 -
 .../kafka/internals/ZookeeperOffsetHandler.java |  128 --
 .../kafka/partitioner/FixedPartitioner.java     |   80 -
 .../kafka/partitioner/KafkaPartitioner.java     |   42 -
 .../KafkaConsumerPartitionAssignmentTest.java   |  257 ----
 .../connectors/kafka/KafkaConsumerTest.java     |  144 --
 .../connectors/kafka/KafkaConsumerTestBase.java | 1124 --------------
 .../streaming/connectors/kafka/KafkaITCase.java |  105 --
 .../connectors/kafka/KafkaLocalSystemTime.java  |   48 -
 .../connectors/kafka/KafkaProducerITCase.java   |  188 ---
 .../connectors/kafka/KafkaProducerTest.java     |  114 --
 .../connectors/kafka/KafkaTestBase.java         |  382 -----
 .../connectors/kafka/TestFixedPartitioner.java  |  104 --
 .../internals/ZookeeperOffsetHandlerTest.java   |   60 -
 .../kafka/testutils/DataGenerators.java         |  214 ---
 .../kafka/testutils/DiscardingSink.java         |   33 -
 .../kafka/testutils/FailingIdentityMapper.java  |  115 --
 .../testutils/JobManagerCommunicationUtils.java |   76 -
 .../kafka/testutils/MockRuntimeContext.java     |  132 --
 .../testutils/PartitionValidatingMapper.java    |   53 -
 .../kafka/testutils/SuccessException.java       |   26 -
 .../kafka/testutils/ThrottledMapper.java        |   44 -
 .../kafka/testutils/Tuple2Partitioner.java      |   51 -
 .../testutils/ValidatingExactlyOnceSink.java    |   81 -
 .../src/test/resources/log4j-test.properties    |   29 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-connector-nifi/pom.xml                |   94 --
 .../connectors/nifi/NiFiDataPacket.java         |   39 -
 .../connectors/nifi/NiFiDataPacketBuilder.java  |   34 -
 .../streaming/connectors/nifi/NiFiSink.java     |   74 -
 .../streaming/connectors/nifi/NiFiSource.java   |  146 --
 .../connectors/nifi/StandardNiFiDataPacket.java |   46 -
 .../nifi/examples/NiFiSinkTopologyExample.java  |   55 -
 .../examples/NiFiSourceTopologyExample.java     |   58 -
 .../src/test/resources/NiFi_Flink.xml           |   16 -
 .../flink-connector-rabbitmq/pom.xml            |   59 -
 .../streaming/connectors/rabbitmq/RMQSink.java  |  111 --
 .../connectors/rabbitmq/RMQSource.java          |  105 --
 .../connectors/rabbitmq/RMQTopology.java        |   52 -
 .../flink-connector-twitter/pom.xml             |   97 --
 .../connectors/json/JSONParseFlatMap.java       |  144 --
 .../streaming/connectors/json/JSONParser.java   |  175 ---
 .../connectors/twitter/TwitterFilterSource.java |  280 ----
 .../twitter/TwitterFilterSourceExample.java     |   68 -
 .../connectors/twitter/TwitterSource.java       |  233 ---
 .../connectors/twitter/TwitterStreaming.java    |   99 --
 .../connectors/twitter/TwitterTopology.java     |   92 --
 .../src/main/resources/twitter.properties       |   19 -
 .../connectors/json/JSONParserTest.java         |   74 -
 .../connectors/json/JSONParserTest2.java        |   95 --
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-streaming-connectors/pom.xml          |   66 -
 .../flink-streaming-core/pom.xml                |  116 --
 .../flink/streaming/api/CheckpointingMode.java  |   75 -
 .../flink/streaming/api/TimeCharacteristic.java |   81 -
 .../api/checkpoint/CheckpointNotifier.java      |   37 -
 .../streaming/api/checkpoint/Checkpointed.java  |   65 -
 .../checkpoint/CheckpointedAsynchronously.java  |   37 -
 .../BroadcastOutputSelectorWrapper.java         |   45 -
 .../selector/DirectedOutputSelectorWrapper.java |   97 --
 .../api/collector/selector/OutputSelector.java  |   44 -
 .../selector/OutputSelectorWrapper.java         |   32 -
 .../selector/OutputSelectorWrapperFactory.java  |   33 -
 .../api/datastream/AllWindowedStream.java       |  556 -------
 .../api/datastream/CoGroupedStreams.java        |  575 -------
 .../api/datastream/ConnectedStreams.java        |  331 ----
 .../streaming/api/datastream/DataStream.java    | 1077 -------------
 .../api/datastream/DataStreamSink.java          |   82 -
 .../api/datastream/DataStreamSource.java        |   53 -
 .../api/datastream/IterativeStream.java         |  218 ---
 .../streaming/api/datastream/JoinedStreams.java |  330 ----
 .../streaming/api/datastream/KeyedStream.java   |  499 ------
 .../datastream/SingleOutputStreamOperator.java  |  309 ----
 .../streaming/api/datastream/SplitStream.java   |   62 -
 .../api/datastream/StreamProjection.java        |  484 ------
 .../api/datastream/WindowedStream.java          |  648 --------
 .../api/environment/LocalStreamEnvironment.java |  109 --
 .../environment/RemoteStreamEnvironment.java    |  253 ---
 .../environment/StreamContextEnvironment.java   |  103 --
 .../environment/StreamExecutionEnvironment.java | 1454 ------------------
 .../StreamExecutionEnvironmentFactory.java      |   31 -
 .../api/environment/StreamPlanEnvironment.java  |   68 -
 .../functions/AscendingTimestampExtractor.java  |   59 -
 .../api/functions/TimestampExtractor.java       |   69 -
 .../aggregation/AggregationFunction.java        |   35 -
 .../aggregation/ComparableAggregator.java       |   89 --
 .../api/functions/aggregation/Comparator.java   |  104 --
 .../functions/aggregation/SumAggregator.java    |   48 -
 .../api/functions/aggregation/SumFunction.java  |  102 --
 .../api/functions/co/CoFlatMapFunction.java     |   42 -
 .../api/functions/co/CoMapFunction.java         |   41 -
 .../api/functions/co/RichCoFlatMapFunction.java |   40 -
 .../api/functions/co/RichCoMapFunction.java     |   40 -
 .../api/functions/sink/FileSinkFunction.java    |  129 --
 .../sink/FileSinkFunctionByMillis.java          |   59 -
 .../api/functions/sink/PrintSinkFunction.java   |   96 --
 .../api/functions/sink/RichSinkFunction.java    |   28 -
 .../api/functions/sink/SinkFunction.java        |   38 -
 .../api/functions/sink/SocketClientSink.java    |  268 ----
 .../api/functions/sink/WriteFormat.java         |   43 -
 .../api/functions/sink/WriteFormatAsCsv.java    |   49 -
 .../api/functions/sink/WriteFormatAsText.java   |   47 -
 .../api/functions/sink/WriteSinkFunction.java   |   92 --
 .../sink/WriteSinkFunctionByMillis.java         |   50 -
 .../api/functions/source/ConnectorSource.java   |   38 -
 .../source/EventTimeSourceFunction.java         |   39 -
 .../source/FileMonitoringFunction.java          |  131 --
 .../api/functions/source/FileReadFunction.java  |   51 -
 .../functions/source/FileSourceFunction.java    |  146 --
 .../functions/source/FromElementsFunction.java  |  196 ---
 .../functions/source/FromIteratorFunction.java  |   45 -
 .../source/FromSplittableIteratorFunction.java  |   58 -
 .../source/MessageAcknowledingSourceBase.java   |  172 ---
 .../source/ParallelSourceFunction.java          |   34 -
 .../source/RichEventTimeSourceFunction.java     |   47 -
 .../source/RichParallelSourceFunction.java      |   38 -
 .../functions/source/RichSourceFunction.java    |   46 -
 .../source/SocketTextStreamFunction.java        |  146 --
 .../api/functions/source/SourceFunction.java    |  162 --
 .../source/StatefulSequenceSource.java          |   86 --
 .../functions/windowing/AllWindowFunction.java  |   45 -
 .../windowing/FoldAllWindowFunction.java        |   97 --
 .../functions/windowing/FoldWindowFunction.java |   97 --
 .../windowing/ReduceAllWindowFunction.java      |   70 -
 .../windowing/ReduceWindowFunction.java         |   50 -
 .../ReduceWindowFunctionWithWindow.java         |   71 -
 .../windowing/RichAllWindowFunction.java        |   25 -
 .../functions/windowing/RichWindowFunction.java |   25 -
 .../api/functions/windowing/WindowFunction.java |   47 -
 .../windowing/delta/CosineDistance.java         |   92 --
 .../windowing/delta/DeltaFunction.java          |   44 -
 .../windowing/delta/EuclideanDistance.java      |   58 -
 .../delta/ExtractionAwareDeltaFunction.java     |   90 --
 .../delta/extractor/ArrayFromTuple.java         |   74 -
 .../delta/extractor/ConcatenatedExtract.java    |   68 -
 .../windowing/delta/extractor/Extractor.java    |   43 -
 .../delta/extractor/FieldFromArray.java         |   59 -
 .../delta/extractor/FieldFromTuple.java         |   58 -
 .../delta/extractor/FieldsFromArray.java        |   67 -
 .../delta/extractor/FieldsFromTuple.java        |   53 -
 .../streaming/api/graph/JSONGenerator.java      |  189 ---
 .../flink/streaming/api/graph/StreamConfig.java |  468 ------
 .../flink/streaming/api/graph/StreamEdge.java   |  120 --
 .../flink/streaming/api/graph/StreamGraph.java  |  619 --------
 .../api/graph/StreamGraphGenerator.java         |  538 -------
 .../flink/streaming/api/graph/StreamNode.java   |  263 ----
 .../api/graph/StreamingJobGraphGenerator.java   |  444 ------
 .../api/operators/AbstractStreamOperator.java   |  373 -----
 .../operators/AbstractUdfStreamOperator.java    |  192 ---
 .../api/operators/ChainingStrategy.java         |   47 -
 .../api/operators/OneInputStreamOperator.java   |   47 -
 .../flink/streaming/api/operators/Output.java   |   40 -
 .../api/operators/OutputTypeConfigurable.java   |   42 -
 .../streaming/api/operators/StreamCounter.java  |   42 -
 .../streaming/api/operators/StreamFilter.java   |   44 -
 .../streaming/api/operators/StreamFlatMap.java  |   53 -
 .../api/operators/StreamGroupedFold.java        |  115 --
 .../api/operators/StreamGroupedReduce.java      |   70 -
 .../streaming/api/operators/StreamMap.java      |   44 -
 .../streaming/api/operators/StreamOperator.java |  146 --
 .../streaming/api/operators/StreamProject.java  |   64 -
 .../streaming/api/operators/StreamSink.java     |   44 -
 .../streaming/api/operators/StreamSource.java   |  300 ----
 .../api/operators/StreamingRuntimeContext.java  |  176 ---
 .../api/operators/TimestampedCollector.java     |   67 -
 .../api/operators/TwoInputStreamOperator.java   |   63 -
 .../api/operators/co/CoStreamFlatMap.java       |   87 --
 .../streaming/api/operators/co/CoStreamMap.java |   71 -
 .../CoFeedbackTransformation.java               |  122 --
 .../transformations/FeedbackTransformation.java |  124 --
 .../transformations/OneInputTransformation.java |  127 --
 .../PartitionTransformation.java                |   82 -
 .../transformations/SelectTransformation.java   |   84 -
 .../api/transformations/SinkTransformation.java |  117 --
 .../transformations/SourceTransformation.java   |   70 -
 .../transformations/SplitTransformation.java    |   84 -
 .../transformations/StreamTransformation.java   |  321 ----
 .../transformations/TwoInputTransformation.java |  116 --
 .../transformations/UnionTransformation.java    |   81 -
 .../streaming/api/watermark/Watermark.java      |   76 -
 .../api/windowing/assigners/GlobalWindows.java  |   96 --
 .../windowing/assigners/SlidingTimeWindows.java |  109 --
 .../assigners/TumblingTimeWindows.java          |   94 --
 .../api/windowing/assigners/WindowAssigner.java |   63 -
 .../api/windowing/evictors/CountEvictor.java    |   54 -
 .../api/windowing/evictors/DeltaEvictor.java    |   74 -
 .../api/windowing/evictors/Evictor.java         |   51 -
 .../api/windowing/evictors/TimeEvictor.java     |   73 -
 .../api/windowing/time/AbstractTime.java        |   98 --
 .../streaming/api/windowing/time/EventTime.java |   62 -
 .../api/windowing/time/ProcessingTime.java      |   63 -
 .../streaming/api/windowing/time/Time.java      |   66 -
 .../triggers/ContinuousEventTimeTrigger.java    |   90 --
 .../ContinuousProcessingTimeTrigger.java        |  106 --
 .../api/windowing/triggers/CountTrigger.java    |   76 -
 .../api/windowing/triggers/DeltaTrigger.java    |   89 --
 .../windowing/triggers/EventTimeTrigger.java    |   62 -
 .../triggers/ProcessingTimeTrigger.java         |   60 -
 .../api/windowing/triggers/PurgingTrigger.java  |   99 --
 .../api/windowing/triggers/Trigger.java         |  120 --
 .../api/windowing/windows/GlobalWindow.java     |  128 --
 .../api/windowing/windows/TimeWindow.java       |  157 --
 .../streaming/api/windowing/windows/Window.java |   31 -
 .../streaming/runtime/io/BarrierBuffer.java     |  320 ----
 .../streaming/runtime/io/BarrierTracker.java    |  205 ---
 .../runtime/io/BlockingQueueBroker.java         |   31 -
 .../streaming/runtime/io/BufferSpiller.java     |  410 -----
 .../runtime/io/CheckpointBarrierHandler.java    |   65 -
 .../streaming/runtime/io/CollectorWrapper.java  |   61 -
 .../streaming/runtime/io/InputGateUtil.java     |   59 -
 .../runtime/io/RecordWriterOutput.java          |  112 --
 .../runtime/io/StreamInputProcessor.java        |  216 ---
 .../runtime/io/StreamRecordWriter.java          |  182 ---
 .../runtime/io/StreamTwoInputProcessor.java     |  290 ----
 .../streaming/runtime/io/StreamingReader.java   |   28 -
 .../operators/BucketStreamSortOperator.java     |  103 --
 .../operators/ExtractTimestampsOperator.java    |   95 --
 .../operators/StreamingOperatorMetrics.java     |   27 -
 .../runtime/operators/Triggerable.java          |   37 -
 .../runtime/operators/package-info.java         |   22 -
 ...ractAlignedProcessingTimeWindowOperator.java |  335 ----
 .../windowing/AbstractKeyedTimePanes.java       |  157 --
 .../windowing/AccumulatingKeyedTimePanes.java   |  133 --
 ...ccumulatingProcessingTimeWindowOperator.java |  163 --
 .../windowing/AggregatingKeyedTimePanes.java    |  106 --
 ...AggregatingProcessingTimeWindowOperator.java |   50 -
 .../EvictingNonKeyedWindowOperator.java         |   92 --
 .../windowing/EvictingWindowOperator.java       |   98 --
 .../runtime/operators/windowing/KeyMap.java     |  651 --------
 .../windowing/NonKeyedWindowOperator.java       |  539 -------
 .../operators/windowing/WindowOperator.java     |  625 --------
 .../windowing/buffers/EvictingWindowBuffer.java |   35 -
 .../windowing/buffers/HeapWindowBuffer.java     |   92 --
 .../buffers/PreAggregatingHeapWindowBuffer.java |   98 --
 .../windowing/buffers/WindowBuffer.java         |   64 -
 .../windowing/buffers/WindowBufferFactory.java  |   54 -
 .../operators/windowing/package-info.java       |   22 -
 .../partitioner/BroadcastPartitioner.java       |   60 -
 .../partitioner/CustomPartitionerWrapper.java   |   70 -
 .../runtime/partitioner/ForwardPartitioner.java |   46 -
 .../runtime/partitioner/GlobalPartitioner.java  |   48 -
 .../runtime/partitioner/HashPartitioner.java    |   63 -
 .../partitioner/RebalancePartitioner.java       |   49 -
 .../runtime/partitioner/ShufflePartitioner.java |   55 -
 .../runtime/partitioner/StreamPartitioner.java  |   30 -
 .../MultiplexingStreamRecordSerializer.java     |  188 ---
 .../runtime/streamrecord/StreamElement.java     |   62 -
 .../runtime/streamrecord/StreamRecord.java      |  124 --
 .../streamrecord/StreamRecordSerializer.java    |  146 --
 .../ExceptionInChainedOperatorException.java    |   45 -
 .../runtime/tasks/OneInputStreamTask.java       |   77 -
 .../streaming/runtime/tasks/OperatorChain.java  |  326 ----
 .../runtime/tasks/SourceStreamTask.java         |  106 --
 .../runtime/tasks/StreamIterationHead.java      |  129 --
 .../runtime/tasks/StreamIterationTail.java      |   92 --
 .../streaming/runtime/tasks/StreamTask.java     |  616 --------
 .../runtime/tasks/StreamTaskException.java      |   68 -
 .../runtime/tasks/StreamTaskState.java          |  124 --
 .../runtime/tasks/StreamTaskStateList.java      |   60 -
 .../streaming/runtime/tasks/TimerException.java |   35 -
 .../runtime/tasks/TwoInputStreamTask.java       |  101 --
 .../streaming/runtime/tasks/package-info.java   |   27 -
 .../flink/streaming/util/FieldAccessor.java     |  249 ---
 .../streaming/util/keys/KeySelectorUtil.java    |  240 ---
 .../serialization/DeserializationSchema.java    |   49 -
 .../serialization/JavaDefaultStringSchema.java  |   47 -
 .../streaming/util/serialization/RawSchema.java |   52 -
 .../util/serialization/SerializationSchema.java |   40 -
 .../util/serialization/SimpleStringSchema.java  |   47 -
 .../TypeInformationSerializationSchema.java     |  118 --
 .../consumer/StreamTestSingleInputGate.java     |  228 ---
 .../streaming/api/AggregationFunctionTest.java  |  416 -----
 .../api/ChainedRuntimeContextTest.java          |   80 -
 .../flink/streaming/api/CoStreamTest.java       |  132 --
 .../flink/streaming/api/DataStreamTest.java     |  694 ---------
 .../apache/flink/streaming/api/IterateTest.java |  614 --------
 .../flink/streaming/api/OutputSplitterTest.java |  144 --
 .../flink/streaming/api/PartitionerTest.java    |  268 ----
 .../flink/streaming/api/SourceFunctionTest.java |   84 -
 .../api/StreamExecutionEnvironmentTest.java     |  163 --
 .../streaming/api/StreamingOperatorsITCase.java |  230 ---
 .../flink/streaming/api/TypeFillTest.java       |  152 --
 .../api/collector/DirectedOutputTest.java       |  119 --
 .../api/collector/OutputSelectorTest.java       |   59 -
 .../api/complex/ComplexIntegrationTest.java     |  837 ----------
 .../api/functions/FromElementsFunctionTest.java |  265 ----
 .../api/functions/ListSourceContext.java        |   82 -
 .../api/functions/PrintSinkFunctionTest.java    |  124 --
 .../functions/sink/SocketClientSinkTest.java    |  303 ----
 .../source/FileMonitoringFunctionTest.java      |   63 -
 .../source/SocketTextStreamFunctionTest.java    |  349 -----
 .../delta/extractor/ArrayFromTupleTest.java     |  118 --
 .../extractor/ConcatenatedExtractTest.java      |   77 -
 .../delta/extractor/FieldFromArrayTest.java     |   54 -
 .../delta/extractor/FieldFromTupleTest.java     |   83 -
 .../delta/extractor/FieldsFromArrayTest.java    |  108 --
 .../delta/extractor/FieldsFromTupleTest.java    |  106 --
 .../streaming/api/graph/SlotAllocationTest.java |   56 -
 .../api/graph/StreamGraphGeneratorTest.java     |  301 ----
 .../graph/StreamingJobGraphGeneratorTest.java   |   91 --
 .../api/operators/StreamCounterTest.java        |   61 -
 .../api/operators/StreamFilterTest.java         |  134 --
 .../api/operators/StreamFlatMapTest.java        |  143 --
 .../api/operators/StreamGroupedFoldTest.java    |  161 --
 .../api/operators/StreamGroupedReduceTest.java  |  161 --
 .../streaming/api/operators/StreamMapTest.java  |  128 --
 .../api/operators/StreamProjectTest.java        |  137 --
 .../api/operators/co/CoGroupedReduceTest.java   |  125 --
 .../api/operators/co/CoStreamFlatMapTest.java   |  169 --
 .../api/operators/co/CoStreamMapTest.java       |  163 --
 .../api/operators/co/CoWindowTest.java          |  182 ---
 .../api/operators/co/SelfConnectionTest.java    |  180 ---
 .../api/outputformat/CsvOutputFormatITCase.java |   78 -
 .../outputformat/SocketOutputFormatITCase.java  |   52 -
 .../outputformat/TextOutputFormatITCase.java    |   55 -
 .../api/streamtask/MockRecordWriter.java        |   45 -
 .../api/streamtask/StreamIterationHeadTest.java |   51 -
 .../api/streamtask/StreamVertexTest.java        |  187 ---
 .../deltafunction/CosineDistanceTest.java       |   71 -
 .../deltafunction/EuclideanDistanceTest.java    |   72 -
 .../flink/streaming/graph/TranslationTest.java  |   76 -
 .../io/BarrierBufferMassiveRandomTest.java      |  175 ---
 .../streaming/runtime/io/BarrierBufferTest.java |  954 ------------
 .../runtime/io/BarrierTrackerTest.java          |  367 -----
 .../streaming/runtime/io/BufferSpillerTest.java |  407 -----
 .../streaming/runtime/io/MockInputGate.java     |   94 --
 .../io/SpilledBufferOrEventSequenceTest.java    |  482 ------
 .../runtime/io/StreamRecordWriterTest.java      |  131 --
 .../flink/streaming/runtime/io/TestEvent.java   |   88 --
 .../runtime/operators/StreamTaskTimerTest.java  |  175 ---
 ...AlignedProcessingTimeWindowOperatorTest.java |  824 ----------
 ...AlignedProcessingTimeWindowOperatorTest.java |  823 ----------
 .../windowing/AllWindowTranslationTest.java     |  215 ---
 .../operators/windowing/CoGroupJoinITCase.java  |  373 -----
 .../operators/windowing/CollectingOutput.java   |   80 -
 .../EvictingNonKeyedWindowOperatorTest.java     |  173 ---
 .../windowing/EvictingWindowOperatorTest.java   |  182 ---
 .../windowing/KeyMapPutIfAbsentTest.java        |  121 --
 .../operators/windowing/KeyMapPutTest.java      |  136 --
 .../runtime/operators/windowing/KeyMapTest.java |  344 -----
 .../windowing/NonKeyedWindowOperatorTest.java   |  429 ------
 .../windowing/TimeWindowTranslationTest.java    |  143 --
 .../operators/windowing/WindowFoldITCase.java   |  191 ---
 .../operators/windowing/WindowOperatorTest.java |  470 ------
 .../windowing/WindowTranslationTest.java        |  265 ----
 .../partitioner/BroadcastPartitionerTest.java   |   55 -
 .../partitioner/ForwardPartitionerTest.java     |   55 -
 .../partitioner/GlobalPartitionerTest.java      |   50 -
 .../partitioner/HashPartitionerTest.java        |   71 -
 .../partitioner/RebalancePartitionerTest.java   |   56 -
 .../partitioner/ShufflePartitionerTest.java     |   60 -
 .../StreamRecordSerializerTest.java             |   68 -
 .../runtime/tasks/OneInputStreamTaskTest.java   |  334 ----
 .../tasks/OneInputStreamTaskTestHarness.java    |  104 --
 .../runtime/tasks/SourceStreamTaskTest.java     |  281 ----
 .../runtime/tasks/StreamMockEnvironment.java    |  304 ----
 .../runtime/tasks/StreamTaskTestHarness.java    |  321 ----
 .../runtime/tasks/StreamTaskTimerITCase.java    |  313 ----
 .../runtime/tasks/TwoInputStreamTaskTest.java   |  373 -----
 .../tasks/TwoInputStreamTaskTestHarness.java    |  170 --
 .../streaming/timestamp/TimestampITCase.java    |  619 --------
 .../streaming/util/EvenOddOutputSelector.java   |   31 -
 .../flink/streaming/util/FieldAccessorTest.java |   75 -
 .../flink/streaming/util/MockContext.java       |  159 --
 .../apache/flink/streaming/util/MockOutput.java |   50 -
 .../apache/flink/streaming/util/NoOpIntMap.java |   28 -
 .../apache/flink/streaming/util/NoOpSink.java   |   26 -
 .../util/OneInputStreamOperatorTestHarness.java |  159 --
 .../streaming/util/ReceiveCheckNoOpSink.java    |   42 -
 .../streaming/util/SocketOutputTestBase.java    |  130 --
 .../streaming/util/SocketProgramITCaseBase.java |   90 --
 .../streaming/util/SourceFunctionUtil.java      |   75 -
 .../util/StreamingMultipleProgramsTestBase.java |   81 -
 .../util/StreamingProgramTestBase.java          |  111 --
 .../flink/streaming/util/TestHarnessUtil.java   |   85 -
 .../streaming/util/TestListResultSink.java      |   74 -
 .../flink/streaming/util/TestListWrapper.java   |   64 -
 .../streaming/util/TestStreamEnvironment.java   |   75 -
 .../util/TwoInputStreamOperatorTestHarness.java |  153 --
 .../TypeInformationSerializationSchemaTest.java |  118 --
 .../util/keys/ArrayKeySelectorTest.java         |   80 -
 .../src/test/resources/log4j-test.properties    |   27 -
 .../src/test/resources/log4j.properties         |   27 -
 .../src/test/resources/logback-test.xml         |   30 -
 .../flink-streaming-examples/pom.xml            |  535 -------
 .../examples/iteration/IterateExample.java      |  247 ---
 .../iteration/util/IterateExampleData.java      |   32 -
 .../streaming/examples/join/WindowJoin.java     |  296 ----
 .../examples/join/util/WindowJoinData.java      |   61 -
 .../ml/IncrementalLearningSkeleton.java         |  255 ---
 .../util/IncrementalLearningSkeletonData.java   |   32 -
 .../socket/SocketTextStreamWordCount.java       |  108 --
 .../examples/twitter/TwitterStream.java         |  167 --
 .../twitter/util/TwitterStreamData.java         |   32 -
 .../GroupedProcessingTimeWindowExample.java     |  127 --
 .../examples/windowing/SessionWindowing.java    |  168 --
 .../examples/windowing/TopSpeedWindowing.java   |  210 ---
 .../examples/windowing/WindowWordCount.java     |  132 --
 .../windowing/util/SessionWindowingData.java    |   27 -
 .../util/TopSpeedWindowingExampleData.java      |  276 ----
 .../examples/wordcount/PojoExample.java         |  186 ---
 .../streaming/examples/wordcount/WordCount.java |  148 --
 .../scala/examples/join/WindowJoin.scala        |  156 --
 .../socket/SocketTextStreamWordCount.scala      |   93 --
 .../examples/windowing/TopSpeedWindowing.scala  |  150 --
 .../iteration/IterateExampleITCase.java         |   45 -
 .../join/WindowJoinITCase.java                  |   50 -
 .../ml/IncrementalLearningSkeletonITCase.java   |   42 -
 .../socket/SocketTextStreamWordCountITCase.java |   30 -
 .../twitter/TwitterStreamITCase.java            |   42 -
 .../windowing/SessionWindowingITCase.java       |   42 -
 .../TopSpeedWindowingExampleITCase.java         |   45 -
 .../windowing/WindowWordCountITCase.java        |   50 -
 .../wordcount/PojoExampleITCase.java            |   45 -
 .../wordcount/WordCountITCase.java              |   45 -
 .../join/WindowJoinITCase.java                  |   50 -
 .../socket/SocketTextStreamWordCountITCase.java |   30 -
 .../TopSpeedWindowingExampleITCase.java         |   45 -
 .../flink-streaming-scala/pom.xml               |  236 ---
 .../streaming/api/scala/AllWindowedStream.scala |  364 -----
 .../streaming/api/scala/CoGroupedStreams.scala  |  303 ----
 .../streaming/api/scala/ConnectedStreams.scala  |  364 -----
 .../flink/streaming/api/scala/DataStream.scala  |  767 ---------
 .../streaming/api/scala/JoinedStreams.scala     |  313 ----
 .../flink/streaming/api/scala/KeyedStream.scala |  370 -----
 .../flink/streaming/api/scala/SplitStream.scala |   37 -
 .../api/scala/StreamExecutionEnvironment.scala  |  657 --------
 .../streaming/api/scala/WindowedStream.scala    |  371 -----
 .../api/scala/function/StatefulFunction.scala   |   48 -
 .../flink/streaming/api/scala/package.scala     |   76 -
 .../scala/api/CsvOutputFormatITCase.java        |   67 -
 .../scala/api/SocketOutputFormatITCase.java     |   36 -
 .../scala/api/StatefulFunctionITCase.java       |   30 -
 .../scala/api/TextOutputFormatITCase.java       |   43 -
 .../api/scala/AllWindowTranslationTest.scala    |  265 ----
 .../streaming/api/scala/CoGroupJoinITCase.scala |  275 ----
 .../streaming/api/scala/DataStreamTest.scala    |  543 -------
 .../api/scala/OutputFormatTestPrograms.scala    |   78 -
 ...ScalaStreamingMultipleProgramsTestBase.scala |   55 -
 .../streaming/api/scala/StateTestPrograms.scala |   94 --
 .../api/scala/StreamingOperatorsITCase.scala    |  116 --
 .../StreamingScalaAPICompletenessTest.scala     |  153 --
 .../streaming/api/scala/WindowFoldITCase.scala  |  148 --
 .../api/scala/WindowTranslationTest.scala       |  255 ---
 flink-staging/flink-streaming/pom.xml           |   43 -
 flink-staging/pom.xml                           |    1 -
 .../flink-connector-elasticsearch/pom.xml       |  106 ++
 .../elasticsearch/ElasticsearchSink.java        |  315 ++++
 .../elasticsearch/IndexRequestBuilder.java      |   66 +
 .../examples/ElasticsearchExample.java          |   81 +
 .../elasticsearch/ElasticsearchSinkITCase.java  |  205 +++
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-filesystem/pom.xml          |  112 ++
 .../flink/streaming/connectors/fs/Bucketer.java |   52 +
 .../flink/streaming/connectors/fs/Clock.java    |   32 +
 .../connectors/fs/DateTimeBucketer.java         |  124 ++
 .../connectors/fs/NonRollingBucketer.java       |   43 +
 .../streaming/connectors/fs/RollingSink.java    |  900 +++++++++++
 .../connectors/fs/SequenceFileWriter.java       |  160 ++
 .../streaming/connectors/fs/StringWriter.java   |  103 ++
 .../streaming/connectors/fs/SystemClock.java    |   28 +
 .../flink/streaming/connectors/fs/Writer.java   |   64 +
 .../src/main/resources/log4j.properties         |   27 +
 .../fs/RollingSinkFaultTolerance2ITCase.java    |  289 ++++
 .../fs/RollingSinkFaultToleranceITCase.java     |  284 ++++
 .../connectors/fs/RollingSinkITCase.java        |  506 ++++++
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-flume/pom.xml               |  174 +++
 .../streaming/connectors/flume/FlumeSink.java   |  141 ++
 .../streaming/connectors/flume/FlumeSource.java |  149 ++
 .../connectors/flume/FlumeTopology.java         |   49 +
 .../flink-connector-kafka/pom.xml               |  130 ++
 .../connectors/kafka/FlinkKafkaConsumer.java    |  689 +++++++++
 .../connectors/kafka/FlinkKafkaConsumer081.java |   57 +
 .../connectors/kafka/FlinkKafkaConsumer082.java |   51 +
 .../connectors/kafka/FlinkKafkaProducer.java    |  288 ++++
 .../connectors/kafka/api/KafkaSink.java         |   34 +
 .../api/persistent/PersistentKafkaSource.java   |   52 +
 .../connectors/kafka/internals/Fetcher.java     |   83 +
 .../kafka/internals/LegacyFetcher.java          |  622 ++++++++
 .../kafka/internals/OffsetHandler.java          |   58 +
 .../kafka/internals/PartitionerWrapper.java     |   49 +
 .../internals/ZooKeeperStringSerializer.java    |   51 +
 .../kafka/internals/ZookeeperOffsetHandler.java |  128 ++
 .../kafka/partitioner/FixedPartitioner.java     |   80 +
 .../kafka/partitioner/KafkaPartitioner.java     |   42 +
 .../KafkaConsumerPartitionAssignmentTest.java   |  257 ++++
 .../connectors/kafka/KafkaConsumerTest.java     |  144 ++
 .../connectors/kafka/KafkaConsumerTestBase.java | 1124 ++++++++++++++
 .../streaming/connectors/kafka/KafkaITCase.java |  105 ++
 .../connectors/kafka/KafkaLocalSystemTime.java  |   48 +
 .../connectors/kafka/KafkaProducerITCase.java   |  188 +++
 .../connectors/kafka/KafkaProducerTest.java     |  114 ++
 .../connectors/kafka/KafkaTestBase.java         |  382 +++++
 .../connectors/kafka/TestFixedPartitioner.java  |  104 ++
 .../internals/ZookeeperOffsetHandlerTest.java   |   60 +
 .../kafka/testutils/DataGenerators.java         |  214 +++
 .../kafka/testutils/DiscardingSink.java         |   33 +
 .../kafka/testutils/FailingIdentityMapper.java  |  115 ++
 .../testutils/JobManagerCommunicationUtils.java |   76 +
 .../kafka/testutils/MockRuntimeContext.java     |  132 ++
 .../testutils/PartitionValidatingMapper.java    |   53 +
 .../kafka/testutils/SuccessException.java       |   26 +
 .../kafka/testutils/ThrottledMapper.java        |   44 +
 .../kafka/testutils/Tuple2Partitioner.java      |   51 +
 .../testutils/ValidatingExactlyOnceSink.java    |   81 +
 .../src/test/resources/log4j-test.properties    |   29 +
 .../src/test/resources/logback-test.xml         |   30 +
 .../flink-connector-nifi/pom.xml                |   94 ++
 .../connectors/nifi/NiFiDataPacket.java         |   39 +
 .../connectors/nifi/NiFiDataPacketBuilder.java  |   34 +
 .../streaming/connectors/nifi/NiFiSink.java     |   74 +
 .../streaming/connectors/nifi/NiFiSource.java   |  146 ++
 .../connectors/nifi/StandardNiFiDataPacket.java |   46 +
 .../nifi/examples/NiFiSinkTopologyExample.java  |   55 +
 .../examples/NiFiSourceTopologyExample.java     |   58 +
 .../src/test/resources/NiFi_Flink.xml           |   16 +
 .../flink-connector-rabbitmq/pom.xml            |   59 +
 .../streaming/connectors/rabbitmq/RMQSink.java  |  111 ++
 .../connectors/rabbitmq/RMQSource.java          |  105 ++
 .../connectors/rabbitmq/RMQTopology.java        |   52 +
 .../flink-connector-twitter/pom.xml             |   97 ++
 .../connectors/json/JSONParseFlatMap.java       |  144 ++
 .../streaming/connectors/json/JSONParser.java   |  175 +++
 .../connectors/twitter/TwitterFilterSource.java |  280 ++++
 .../twitter/TwitterFilterSourceExample.java     |   68 +
 .../connectors/twitter/TwitterSource.java       |  233 +++
 .../connectors/twitter/TwitterStreaming.java    |   99 ++
 .../connectors/twitter/TwitterTopology.java     |   92 ++
 .../src/main/resources/twitter.properties       |   19 +
 .../connectors/json/JSONParserTest.java         |   74 +
 .../connectors/json/JSONParserTest2.java        |   95 ++
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/logback-test.xml         |   30 +
 flink-streaming-connectors/pom.xml              |   66 +
 flink-streaming-examples/pom.xml                |  535 +++++++
 .../examples/iteration/IterateExample.java      |  247 +++
 .../iteration/util/IterateExampleData.java      |   32 +
 .../streaming/examples/join/WindowJoin.java     |  296 ++++
 .../examples/join/util/WindowJoinData.java      |   61 +
 .../ml/IncrementalLearningSkeleton.java         |  255 +++
 .../util/IncrementalLearningSkeletonData.java   |   32 +
 .../socket/SocketTextStreamWordCount.java       |  108 ++
 .../examples/twitter/TwitterStream.java         |  167 ++
 .../twitter/util/TwitterStreamData.java         |   32 +
 .../GroupedProcessingTimeWindowExample.java     |  127 ++
 .../examples/windowing/SessionWindowing.java    |  168 ++
 .../examples/windowing/TopSpeedWindowing.java   |  210 +++
 .../examples/windowing/WindowWordCount.java     |  132 ++
 .../windowing/util/SessionWindowingData.java    |   27 +
 .../util/TopSpeedWindowingExampleData.java      |  276 ++++
 .../examples/wordcount/PojoExample.java         |  186 +++
 .../streaming/examples/wordcount/WordCount.java |  148 ++
 .../scala/examples/join/WindowJoin.scala        |  156 ++
 .../socket/SocketTextStreamWordCount.scala      |   93 ++
 .../examples/windowing/TopSpeedWindowing.scala  |  150 ++
 .../iteration/IterateExampleITCase.java         |   45 +
 .../join/WindowJoinITCase.java                  |   50 +
 .../ml/IncrementalLearningSkeletonITCase.java   |   42 +
 .../socket/SocketTextStreamWordCountITCase.java |   30 +
 .../twitter/TwitterStreamITCase.java            |   42 +
 .../windowing/SessionWindowingITCase.java       |   42 +
 .../TopSpeedWindowingExampleITCase.java         |   45 +
 .../windowing/WindowWordCountITCase.java        |   50 +
 .../wordcount/PojoExampleITCase.java            |   45 +
 .../wordcount/WordCountITCase.java              |   45 +
 .../join/WindowJoinITCase.java                  |   50 +
 .../socket/SocketTextStreamWordCountITCase.java |   30 +
 .../TopSpeedWindowingExampleITCase.java         |   45 +
 flink-streaming-java/pom.xml                    |  116 ++
 .../flink/streaming/api/CheckpointingMode.java  |   75 +
 .../flink/streaming/api/TimeCharacteristic.java |   81 +
 .../api/checkpoint/CheckpointNotifier.java      |   37 +
 .../streaming/api/checkpoint/Checkpointed.java  |   65 +
 .../checkpoint/CheckpointedAsynchronously.java  |   37 +
 .../BroadcastOutputSelectorWrapper.java         |   45 +
 .../selector/DirectedOutputSelectorWrapper.java |   97 ++
 .../api/collector/selector/OutputSelector.java  |   44 +
 .../selector/OutputSelectorWrapper.java         |   32 +
 .../selector/OutputSelectorWrapperFactory.java  |   33 +
 .../api/datastream/AllWindowedStream.java       |  556 +++++++
 .../api/datastream/CoGroupedStreams.java        |  575 +++++++
 .../api/datastream/ConnectedStreams.java        |  331 ++++
 .../streaming/api/datastream/DataStream.java    | 1077 +++++++++++++
 .../api/datastream/DataStreamSink.java          |   82 +
 .../api/datastream/DataStreamSource.java        |   53 +
 .../api/datastream/IterativeStream.java         |  218 +++
 .../streaming/api/datastream/JoinedStreams.java |  330 ++++
 .../streaming/api/datastream/KeyedStream.java   |  499 ++++++
 .../datastream/SingleOutputStreamOperator.java  |  309 ++++
 .../streaming/api/datastream/SplitStream.java   |   62 +
 .../api/datastream/StreamProjection.java        |  484 ++++++
 .../api/datastream/WindowedStream.java          |  648 ++++++++
 .../api/environment/LocalStreamEnvironment.java |  109 ++
 .../environment/RemoteStreamEnvironment.java    |  253 +++
 .../environment/StreamContextEnvironment.java   |  103 ++
 .../environment/StreamExecutionEnvironment.java | 1454 ++++++++++++++++++
 .../StreamExecutionEnvironmentFactory.java      |   31 +
 .../api/environment/StreamPlanEnvironment.java  |   68 +
 .../functions/AscendingTimestampExtractor.java  |   59 +
 .../api/functions/TimestampExtractor.java       |   69 +
 .../aggregation/AggregationFunction.java        |   35 +
 .../aggregation/ComparableAggregator.java       |   89 ++
 .../api/functions/aggregation/Comparator.java   |  104 ++
 .../functions/aggregation/SumAggregator.java    |   48 +
 .../api/functions/aggregation/SumFunction.java  |  102 ++
 .../api/functions/co/CoFlatMapFunction.java     |   42 +
 .../api/functions/co/CoMapFunction.java         |   41 +
 .../api/functions/co/RichCoFlatMapFunction.java |   40 +
 .../api/functions/co/RichCoMapFunction.java     |   40 +
 .../api/functions/sink/FileSinkFunction.java    |  129 ++
 .../sink/FileSinkFunctionByMillis.java          |   59 +
 .../api/functions/sink/PrintSinkFunction.java   |   96 ++
 .../api/functions/sink/RichSinkFunction.java    |   28 +
 .../api/functions/sink/SinkFunction.java        |   38 +
 .../api/functions/sink/SocketClientSink.java    |  268 ++++
 .../api/functions/sink/WriteFormat.java         |   43 +
 .../api/functions/sink/WriteFormatAsCsv.java    |   49 +
 .../api/functions/sink/WriteFormatAsText.java   |   47 +
 .../api/functions/sink/WriteSinkFunction.java   |   92 ++
 .../sink/WriteSinkFunctionByMillis.java         |   50 +
 .../api/functions/source/ConnectorSource.java   |   38 +
 .../source/EventTimeSourceFunction.java         |   39 +
 .../source/FileMonitoringFunction.java          |  131 ++
 .../api/functions/source/FileReadFunction.java  |   51 +
 .../functions/source/FileSourceFunction.java    |  146 ++
 .../functions/source/FromElementsFunction.java  |  196 +++
 .../functions/source/FromIteratorFunction.java  |   45 +
 .../source/FromSplittableIteratorFunction.java  |   58 +
 .../source/MessageAcknowledingSourceBase.java   |  172 +++
 .../source/ParallelSourceFunction.java          |   34 +
 .../source/RichEventTimeSourceFunction.java     |   47 +
 .../source/RichParallelSourceFunction.java      |   38 +
 .../functions/source/RichSourceFunction.java    |   46 +
 .../source/SocketTextStreamFunction.java        |  146 ++
 .../api/functions/source/SourceFunction.java    |  162 ++
 .../source/StatefulSequenceSource.java          |   86 ++
 .../functions/windowing/AllWindowFunction.java  |   45 +
 .../windowing/FoldAllWindowFunction.java        |   97 ++
 .../functions/windowing/FoldWindowFunction.java |   97 ++
 .../windowing/ReduceAllWindowFunction.java      |   70 +
 .../windowing/ReduceWindowFunction.java         |   50 +
 .../ReduceWindowFunctionWithWindow.java         |   71 +
 .../windowing/RichAllWindowFunction.java        |   25 +
 .../functions/windowing/RichWindowFunction.java |   25 +
 .../api/functions/windowing/WindowFunction.java |   47 +
 .../windowing/delta/CosineDistance.java         |   92 ++
 .../windowing/delta/DeltaFunction.java          |   44 +
 .../windowing/delta/EuclideanDistance.java      |   58 +
 .../delta/ExtractionAwareDeltaFunction.java     |   90 ++
 .../delta/extractor/ArrayFromTuple.java         |   74 +
 .../delta/extractor/ConcatenatedExtract.java    |   68 +
 .../windowing/delta/extractor/Extractor.java    |   43 +
 .../delta/extractor/FieldFromArray.java         |   59 +
 .../delta/extractor/FieldFromTuple.java         |   58 +
 .../delta/extractor/FieldsFromArray.java        |   67 +
 .../delta/extractor/FieldsFromTuple.java        |   53 +
 .../streaming/api/graph/JSONGenerator.java      |  189 +++
 .../flink/streaming/api/graph/StreamConfig.java |  468 ++++++
 .../flink/streaming/api/graph/StreamEdge.java   |  120 ++
 .../flink/streaming/api/graph/StreamGraph.java  |  619 ++++++++
 .../api/graph/StreamGraphGenerator.java         |  538 +++++++
 .../flink/streaming/api/graph/StreamNode.java   |  263 ++++
 .../api/graph/StreamingJobGraphGenerator.java   |  444 ++++++
 .../api/operators/AbstractStreamOperator.java   |  373 +++++
 .../operators/AbstractUdfStreamOperator.java    |  192 +++
 .../api/operators/ChainingStrategy.java         |   47 +
 .../api/operators/OneInputStreamOperator.java   |   47 +
 .../flink/streaming/api/operators/Output.java   |   40 +
 .../api/operators/OutputTypeConfigurable.java   |   42 +
 .../streaming/api/operators/StreamCounter.java  |   42 +
 .../streaming/api/operators/StreamFilter.java   |   44 +
 .../streaming/api/operators/StreamFlatMap.java  |   53 +
 .../api/operators/StreamGroupedFold.java        |  115 ++
 .../api/operators/StreamGroupedReduce.java      |   70 +
 .../streaming/api/operators/StreamMap.java      |   44 +
 .../streaming/api/operators/StreamOperator.java |  146 ++
 .../streaming/api/operators/StreamProject.java  |   64 +
 .../streaming/api/operators/StreamSink.java     |   44 +
 .../streaming/api/operators/StreamSource.java   |  300 ++++
 .../api/operators/StreamingRuntimeContext.java  |  176 +++
 .../api/operators/TimestampedCollector.java     |   67 +
 .../api/operators/TwoInputStreamOperator.java   |   63 +
 .../api/operators/co/CoStreamFlatMap.java       |   87 ++
 .../streaming/api/operators/co/CoStreamMap.java |   71 +
 .../CoFeedbackTransformation.java               |  122 ++
 .../transformations/FeedbackTransformation.java |  124 ++
 .../transformations/OneInputTransformation.java |  127 ++
 .../PartitionTransformation.java                |   82 +
 .../transformations/SelectTransformation.java   |   84 +
 .../api/transformations/SinkTransformation.java |  117 ++
 .../transformations/SourceTransformation.java   |   70 +
 .../transformations/SplitTransformation.java    |   84 +
 .../transformations/StreamTransformation.java   |  321 ++++
 .../transformations/TwoInputTransformation.java |  116 ++
 .../transformations/UnionTransformation.java    |   81 +
 .../streaming/api/watermark/Watermark.java      |   76 +
 .../api/windowing/assigners/GlobalWindows.java  |   96 ++
 .../windowing/assigners/SlidingTimeWindows.java |  109 ++
 .../assigners/TumblingTimeWindows.java          |   94 ++
 .../api/windowing/assigners/WindowAssigner.java |   63 +
 .../api/windowing/evictors/CountEvictor.java    |   54 +
 .../api/windowing/evictors/DeltaEvictor.java    |   74 +
 .../api/windowing/evictors/Evictor.java         |   51 +
 .../api/windowing/evictors/TimeEvictor.java     |   73 +
 .../api/windowing/time/AbstractTime.java        |   98 ++
 .../streaming/api/windowing/time/EventTime.java |   62 +
 .../api/windowing/time/ProcessingTime.java      |   63 +
 .../streaming/api/windowing/time/Time.java      |   66 +
 .../triggers/ContinuousEventTimeTrigger.java    |   90 ++
 .../ContinuousProcessingTimeTrigger.java        |  106 ++
 .../api/windowing/triggers/CountTrigger.java    |   76 +
 .../api/windowing/triggers/DeltaTrigger.java    |   89 ++
 .../windowing/triggers/EventTimeTrigger.java    |   62 +
 .../triggers/ProcessingTimeTrigger.java         |   60 +
 .../api/windowing/triggers/PurgingTrigger.java  |   99 ++
 .../api/windowing/triggers/Trigger.java         |  120 ++
 .../api/windowing/windows/GlobalWindow.java     |  128 ++
 .../api/windowing/windows/TimeWindow.java       |  157 ++
 .../streaming/api/windowing/windows/Window.java |   31 +
 .../streaming/runtime/io/BarrierBuffer.java     |  320 ++++
 .../streaming/runtime/io/BarrierTracker.java    |  205 +++
 .../runtime/io/BlockingQueueBroker.java         |   31 +
 .../streaming/runtime/io/BufferSpiller.java     |  410 +++++
 .../runtime/io/CheckpointBarrierHandler.java    |   65 +
 .../streaming/runtime/io/CollectorWrapper.java  |   61 +
 .../streaming/runtime/io/InputGateUtil.java     |   59 +
 .../runtime/io/RecordWriterOutput.java          |  112 ++
 .../runtime/io/StreamInputProcessor.java        |  216 +++
 .../runtime/io/StreamRecordWriter.java          |  182 +++
 .../runtime/io/StreamTwoInputProcessor.java     |  290 ++++
 .../streaming/runtime/io/StreamingReader.java   |   28 +
 .../operators/BucketStreamSortOperator.java     |  103 ++
 .../operators/ExtractTimestampsOperator.java    |   95 ++
 .../operators/StreamingOperatorMetrics.java     |   27 +
 .../runtime/operators/Triggerable.java          |   37 +
 .../runtime/operators/package-info.java         |   22 +
 ...ractAlignedProcessingTimeWindowOperator.java |  335 ++++
 .../windowing/AbstractKeyedTimePanes.java       |  157 ++
 .../windowing/AccumulatingKeyedTimePanes.java   |  133 ++
 ...ccumulatingProcessingTimeWindowOperator.java |  163 ++
 .../windowing/AggregatingKeyedTimePanes.java    |  106 ++
 ...AggregatingProcessingTimeWindowOperator.java |   50 +
 .../EvictingNonKeyedWindowOperator.java         |   92 ++
 .../windowing/EvictingWindowOperator.java       |   98 ++
 .../runtime/operators/windowing/KeyMap.java     |  651 ++++++++
 .../windowing/NonKeyedWindowOperator.java       |  539 +++++++
 .../operators/windowing/WindowOperator.java     |  625 ++++++++
 .../windowing/buffers/EvictingWindowBuffer.java |   35 +
 .../windowing/buffers/HeapWindowBuffer.java     |   92 ++
 .../buffers/PreAggregatingHeapWindowBuffer.java |   98 ++
 .../windowing/buffers/WindowBuffer.java         |   64 +
 .../windowing/buffers/WindowBufferFactory.java  |   54 +
 .../operators/windowing/package-info.java       |   22 +
 .../partitioner/BroadcastPartitioner.java       |   60 +
 .../partitioner/CustomPartitionerWrapper.java   |   70 +
 .../runtime/partitioner/ForwardPartitioner.java |   46 +
 .../runtime/partitioner/GlobalPartitioner.java  |   48 +
 .../runtime/partitioner/HashPartitioner.java    |   63 +
 .../partitioner/RebalancePartitioner.java       |   49 +
 .../runtime/partitioner/ShufflePartitioner.java |   55 +
 .../runtime/partitioner/StreamPartitioner.java  |   30 +
 .../MultiplexingStreamRecordSerializer.java     |  188 +++
 .../runtime/streamrecord/StreamElement.java     |   62 +
 .../runtime/streamrecord/StreamRecord.java      |  124 ++
 .../streamrecord/StreamRecordSerializer.java    |  146 ++
 .../ExceptionInChainedOperatorException.java    |   45 +
 .../runtime/tasks/OneInputStreamTask.java       |   77 +
 .../streaming/runtime/tasks/OperatorChain.java  |  326 ++++
 .../runtime/tasks/SourceStreamTask.java         |  106 ++
 .../runtime/tasks/StreamIterationHead.java      |  129 ++
 .../runtime/tasks/StreamIterationTail.java      |   92 ++
 .../streaming/runtime/tasks/StreamTask.java     |  616 ++++++++
 .../runtime/tasks/StreamTaskException.java      |   68 +
 .../runtime/tasks/StreamTaskState.java          |  124 ++
 .../runtime/tasks/StreamTaskStateList.java      |   60 +
 .../streaming/runtime/tasks/TimerException.java |   35 +
 .../runtime/tasks/TwoInputStreamTask.java       |  101 ++
 .../streaming/runtime/tasks/package-info.java   |   27 +
 .../flink/streaming/util/FieldAccessor.java     |  249 +++
 .../streaming/util/keys/KeySelectorUtil.java    |  240 +++
 .../serialization/DeserializationSchema.java    |   49 +
 .../serialization/JavaDefaultStringSchema.java  |   47 +
 .../streaming/util/serialization/RawSchema.java |   52 +
 .../util/serialization/SerializationSchema.java |   40 +
 .../util/serialization/SimpleStringSchema.java  |   47 +
 .../TypeInformationSerializationSchema.java     |  118 ++
 .../consumer/StreamTestSingleInputGate.java     |  228 +++
 .../streaming/api/AggregationFunctionTest.java  |  416 +++++
 .../api/ChainedRuntimeContextTest.java          |   80 +
 .../flink/streaming/api/CoStreamTest.java       |  132 ++
 .../flink/streaming/api/DataStreamTest.java     |  694 +++++++++
 .../apache/flink/streaming/api/IterateTest.java |  614 ++++++++
 .../flink/streaming/api/OutputSplitterTest.java |  144 ++
 .../flink/streaming/api/PartitionerTest.java    |  268 ++++
 .../flink/streaming/api/SourceFunctionTest.java |   84 +
 .../api/StreamExecutionEnvironmentTest.java     |  163 ++
 .../streaming/api/StreamingOperatorsITCase.java |  230 +++
 .../flink/streaming/api/TypeFillTest.java       |  152 ++
 .../api/collector/DirectedOutputTest.java       |  119 ++
 .../api/collector/OutputSelectorTest.java       |   59 +
 .../api/complex/ComplexIntegrationTest.java     |  837 ++++++++++
 .../api/functions/FromElementsFunctionTest.java |  265 ++++
 .../api/functions/ListSourceContext.java        |   82 +
 .../api/functions/PrintSinkFunctionTest.java    |  124 ++
 .../functions/sink/SocketClientSinkTest.java    |  303 ++++
 .../source/FileMonitoringFunctionTest.java      |   63 +
 .../source/SocketTextStreamFunctionTest.java    |  349 +++++
 .../delta/extractor/ArrayFromTupleTest.java     |  118 ++
 .../extractor/ConcatenatedExtractTest.java      |   77 +
 .../delta/extractor/FieldFromArrayTest.java     |   54 +
 .../delta/extractor/FieldFromTupleTest.java     |   83 +
 .../delta/extractor/FieldsFromArrayTest.java    |  108 ++
 .../delta/extractor/FieldsFromTupleTest.java    |  106 ++
 .../streaming/api/graph/SlotAllocationTest.java |   56 +
 .../api/graph/StreamGraphGeneratorTest.java     |  301 ++++
 .../graph/StreamingJobGraphGeneratorTest.java   |   91 ++
 .../api/operators/StreamCounterTest.java        |   61 +
 .../api/operators/StreamFilterTest.java         |  134 ++
 .../api/operators/StreamFlatMapTest.java        |  143 ++
 .../api/operators/StreamGroupedFoldTest.java    |  161 ++
 .../api/operators/StreamGroupedReduceTest.java  |  161 ++
 .../streaming/api/operators/StreamMapTest.java  |  128 ++
 .../api/operators/StreamProjectTest.java        |  137 ++
 .../api/operators/co/CoGroupedReduceTest.java   |  125 ++
 .../api/operators/co/CoStreamFlatMapTest.java   |  169 ++
 .../api/operators/co/CoStreamMapTest.java       |  163 ++
 .../api/operators/co/CoWindowTest.java          |  182 +++
 .../api/operators/co/SelfConnectionTest.java    |  180 +++
 .../api/outputformat/CsvOutputFormatITCase.java |   78 +
 .../outputformat/SocketOutputFormatITCase.java  |   52 +
 .../outputformat/TextOutputFormatITCase.java    |   55 +
 .../api/streamtask/MockRecordWriter.java        |   45 +
 .../api/streamtask/StreamIterationHeadTest.java |   51 +
 .../api/streamtask/StreamVertexTest.java        |  187 +++
 .../deltafunction/CosineDistanceTest.java       |   71 +
 .../deltafunction/EuclideanDistanceTest.java    |   72 +
 .../flink/streaming/graph/TranslationTest.java  |   76 +
 .../io/BarrierBufferMassiveRandomTest.java      |  175 +++
 .../streaming/runtime/io/BarrierBufferTest.java |  954 ++++++++++++
 .../runtime/io/BarrierTrackerTest.java          |  367 +++++
 .../streaming/runtime/io/BufferSpillerTest.java |  407 +++++
 .../streaming/runtime/io/MockInputGate.java     |   94 ++
 .../io/SpilledBufferOrEventSequenceTest.java    |  482 ++++++
 .../runtime/io/StreamRecordWriterTest.java      |  131 ++
 .../flink/streaming/runtime/io/TestEvent.java   |   88 ++
 .../runtime/operators/StreamTaskTimerTest.java  |  175 +++
 ...AlignedProcessingTimeWindowOperatorTest.java |  824 ++++++++++
 ...AlignedProcessingTimeWindowOperatorTest.java |  823 ++++++++++
 .../windowing/AllWindowTranslationTest.java     |  215 +++
 .../operators/windowing/CoGroupJoinITCase.java  |  373 +++++
 .../operators/windowing/CollectingOutput.java   |   80 +
 .../EvictingNonKeyedWindowOperatorTest.java     |  173 +++
 .../windowing/EvictingWindowOperatorTest.java   |  182 +++
 .../windowing/KeyMapPutIfAbsentTest.java        |  121 ++
 .../operators/windowing/KeyMapPutTest.java      |  136 ++
 .../runtime/operators/windowing/KeyMapTest.java |  344 +++++
 .../windowing/NonKeyedWindowOperatorTest.java   |  429 ++++++
 .../windowing/TimeWindowTranslationTest.java    |  143 ++
 .../operators/windowing/WindowFoldITCase.java   |  191 +++
 .../operators/windowing/WindowOperatorTest.java |  470 ++++++
 .../windowing/WindowTranslationTest.java        |  265 ++++
 .../partitioner/BroadcastPartitionerTest.java   |   55 +
 .../partitioner/ForwardPartitionerTest.java     |   55 +
 .../partitioner/GlobalPartitionerTest.java      |   50 +
 .../partitioner/HashPartitionerTest.java        |   71 +
 .../partitioner/RebalancePartitionerTest.java   |   56 +
 .../partitioner/ShufflePartitionerTest.java     |   60 +
 .../StreamRecordSerializerTest.java             |   68 +
 .../runtime/tasks/OneInputStreamTaskTest.java   |  334 ++++
 .../tasks/OneInputStreamTaskTestHarness.java    |  104 ++
 .../runtime/tasks/SourceStreamTaskTest.java     |  281 ++++
 .../runtime/tasks/StreamMockEnvironment.java    |  304 ++++
 .../runtime/tasks/StreamTaskTestHarness.java    |  321 ++++
 .../runtime/tasks/StreamTaskTimerITCase.java    |  313 ++++
 .../runtime/tasks/TwoInputStreamTaskTest.java   |  373 +++++
 .../tasks/TwoInputStreamTaskTestHarness.java    |  170 ++
 .../streaming/timestamp/TimestampITCase.java    |  619 ++++++++
 .../streaming/util/EvenOddOutputSelector.java   |   31 +
 .../flink/streaming/util/FieldAccessorTest.java |   75 +
 .../flink/streaming/util/MockContext.java       |  159 ++
 .../apache/flink/streaming/util/MockOutput.java |   50 +
 .../apache/flink/streaming/util/NoOpIntMap.java |   28 +
 .../apache/flink/streaming/util/NoOpSink.java   |   26 +
 .../util/OneInputStreamOperatorTestHarness.java |  159 ++
 .../streaming/util/ReceiveCheckNoOpSink.java    |   42 +
 .../streaming/util/SocketOutputTestBase.java    |  130 ++
 .../streaming/util/SocketProgramITCaseBase.java |   90 ++
 .../streaming/util/SourceFunctionUtil.java      |   75 +
 .../util/StreamingMultipleProgramsTestBase.java |   81 +
 .../util/StreamingProgramTestBase.java          |  111 ++
 .../flink/streaming/util/TestHarnessUtil.java   |   85 +
 .../streaming/util/TestListResultSink.java      |   74 +
 .../flink/streaming/util/TestListWrapper.java   |   64 +
 .../streaming/util/TestStreamEnvironment.java   |   75 +
 .../util/TwoInputStreamOperatorTestHarness.java |  153 ++
 .../TypeInformationSerializationSchemaTest.java |  118 ++
 .../util/keys/ArrayKeySelectorTest.java         |   80 +
 .../src/test/resources/log4j-test.properties    |   27 +
 .../src/test/resources/log4j.properties         |   27 +
 .../src/test/resources/logback-test.xml         |   30 +
 flink-streaming-scala/pom.xml                   |  236 +++
 .../streaming/api/scala/AllWindowedStream.scala |  364 +++++
 .../streaming/api/scala/CoGroupedStreams.scala  |  303 ++++
 .../streaming/api/scala/ConnectedStreams.scala  |  364 +++++
 .../flink/streaming/api/scala/DataStream.scala  |  767 +++++++++
 .../streaming/api/scala/JoinedStreams.scala     |  313 ++++
 .../flink/streaming/api/scala/KeyedStream.scala |  370 +++++
 .../flink/streaming/api/scala/SplitStream.scala |   37 +
 .../api/scala/StreamExecutionEnvironment.scala  |  657 ++++++++
 .../streaming/api/scala/WindowedStream.scala    |  371 +++++
 .../api/scala/function/StatefulFunction.scala   |   48 +
 .../flink/streaming/api/scala/package.scala     |   76 +
 .../scala/api/CsvOutputFormatITCase.java        |   67 +
 .../scala/api/SocketOutputFormatITCase.java     |   36 +
 .../scala/api/StatefulFunctionITCase.java       |   30 +
 .../scala/api/TextOutputFormatITCase.java       |   43 +
 .../api/scala/AllWindowTranslationTest.scala    |  265 ++++
 .../streaming/api/scala/CoGroupJoinITCase.scala |  275 ++++
 .../streaming/api/scala/DataStreamTest.scala    |  543 +++++++
 .../api/scala/OutputFormatTestPrograms.scala    |   78 +
 ...ScalaStreamingMultipleProgramsTestBase.scala |   55 +
 .../streaming/api/scala/StateTestPrograms.scala |   94 ++
 .../api/scala/StreamingOperatorsITCase.scala    |  116 ++
 .../StreamingScalaAPICompletenessTest.scala     |  153 ++
 .../streaming/api/scala/WindowFoldITCase.scala  |  148 ++
 .../api/scala/WindowTranslationTest.scala       |  255 +++
 flink-tests/pom.xml                             |    2 +-
 flink-yarn-tests/pom.xml                        |    2 +-
 pom.xml                                         |    4 +
 983 files changed, 73316 insertions(+), 73356 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/docs/apis/best_practices.md
----------------------------------------------------------------------
diff --git a/docs/apis/best_practices.md b/docs/apis/best_practices.md
index 3de88f0..9ef519b 100644
--- a/docs/apis/best_practices.md
+++ b/docs/apis/best_practices.md
@@ -342,7 +342,7 @@ Change your projects `pom.xml` file like this:
 	</dependency>
 	<dependency>
 		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-core</artifactId>
+		<artifactId>flink-streaming-java</artifactId>
 		<version>0.10-SNAPSHOT</version>
 		<exclusions>
 			<exclusion>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/docs/apis/streaming_guide.md
----------------------------------------------------------------------
diff --git a/docs/apis/streaming_guide.md b/docs/apis/streaming_guide.md
index 9fce0d7..c655160 100644
--- a/docs/apis/streaming_guide.md
+++ b/docs/apis/streaming_guide.md
@@ -160,7 +160,7 @@ If you want to add Flink to an existing Maven project, add the following entry t
 {% highlight xml %}
 <dependency>
   <groupId>org.apache.flink</groupId>
-  <artifactId>flink-streaming-core</artifactId>
+  <artifactId>flink-streaming-java</artifactId>
   <version>{{site.version }}</version>
 </dependency>
 <dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/docs/internals/stream_checkpointing.md
----------------------------------------------------------------------
diff --git a/docs/internals/stream_checkpointing.md b/docs/internals/stream_checkpointing.md
index 1c8f74f..6bc0f1d 100644
--- a/docs/internals/stream_checkpointing.md
+++ b/docs/internals/stream_checkpointing.md
@@ -141,7 +141,7 @@ It is possible to let an operator continue processing while it stores its state
 After receiving the checkpoint barriers on its inputs, the operator starts the asynchronous snapshot copying of its state. It immediately emits the barrier to its outputs and continues with the regular stream processing. Once the background copy process has completed, it acknowledges the checkpoint to the checkpoint coordinator (the JobManager). The checkpoint is now only complete after all sinks received the barriers and all stateful operators acknowledged their completed backup (which may be later than the barriers reaching the sinks).
 
 User-defined state that is used through the key/value state abstraction can be snapshotted *asynchronously*.
-User functions that implement the interface {% gh_link /flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java "Checkpointed" %} will be snapshotted *synchronously*, while functions that implement {% gh_link /flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java "CheckpointedAsynchronously" %} will be snapshotted *asynchronously*. Note that for the latter, the user function must guarantee that any future modifications to its state to not affect the state object returned by the `snapshotState()` method.
+User functions that implement the interface {% gh_link /flink-staging/flink-streaming/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java "Checkpointed" %} will be snapshotted *synchronously*, while functions that implement {% gh_link /flink-staging/flink-streaming/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java "CheckpointedAsynchronously" %} will be snapshotted *asynchronously*. Note that for the latter, the user function must guarantee that any future modifications to its state to not affect the state object returned by the `snapshotState()` method.
 
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-contrib/flink-storm-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/pom.xml b/flink-contrib/flink-storm-examples/pom.xml
index de6c0cb..696fa4b 100644
--- a/flink-contrib/flink-storm-examples/pom.xml
+++ b/flink-contrib/flink-storm-examples/pom.xml
@@ -49,7 +49,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-contrib/flink-storm/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml
index 657b974..2424b55 100644
--- a/flink-contrib/flink-storm/pom.xml
+++ b/flink-contrib/flink-storm/pom.xml
@@ -37,7 +37,7 @@ under the License.
 	<dependencies>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-contrib/flink-streaming-contrib/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-streaming-contrib/pom.xml b/flink-contrib/flink-streaming-contrib/pom.xml
index 0979643..37ac7ab 100644
--- a/flink-contrib/flink-streaming-contrib/pom.xml
+++ b/flink-contrib/flink-streaming-contrib/pom.xml
@@ -39,7 +39,7 @@ under the License.
 	<dependencies>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 0623cfd..737aeb4 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -97,7 +97,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-java8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-java8/pom.xml b/flink-java8/pom.xml
index 5a81ac5..707c795 100644
--- a/flink-java8/pom.xml
+++ b/flink-java8/pom.xml
@@ -60,7 +60,7 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
index f98002b..24f2199 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
@@ -79,7 +79,7 @@ under the License.
 		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${flink.version}</version>
 		</dependency>
 		<dependency>
@@ -122,7 +122,7 @@ under the License.
 									<exclude>org.apache.flink:flink-java-examples</exclude>
 									<exclude>org.apache.flink:flink-scala-examples</exclude>
 									<exclude>org.apache.flink:flink-streaming-examples</exclude>
-									<exclude>org.apache.flink:flink-streaming-core</exclude>
+									<exclude>org.apache.flink:flink-streaming-java</exclude>
 
 									<!-- Also exclude very big transitive dependencies of Flink
 
@@ -328,7 +328,7 @@ under the License.
 				</dependency>
 				<dependency>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-streaming-core</artifactId>
+					<artifactId>flink-streaming-java</artifactId>
 					<version>${flink.version}</version>
 					<scope>provided</scope>
 				</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
index d365da2..5b86928 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
@@ -126,7 +126,7 @@ under the License.
 									<exclude>org.apache.flink:flink-java-examples</exclude>
 									<exclude>org.apache.flink:flink-scala-examples</exclude>
 									<exclude>org.apache.flink:flink-streaming-examples</exclude>
-									<exclude>org.apache.flink:flink-streaming-core</exclude>
+									<exclude>org.apache.flink:flink-streaming-java</exclude>
 
 									<!-- Also exclude very big transitive dependencies of Flink
 
@@ -334,7 +334,7 @@ under the License.
 				</dependency>
 				<dependency>
 					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-streaming-core</artifactId>
+					<artifactId>flink-streaming-java</artifactId>
 					<version>${flink.version}</version>
 					<scope>provided</scope>
 				</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-fs-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/pom.xml b/flink-staging/flink-fs-tests/pom.xml
index 021d822..4dc86bf 100644
--- a/flink-staging/flink-fs-tests/pom.xml
+++ b/flink-staging/flink-fs-tests/pom.xml
@@ -52,7 +52,7 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-hbase/pom.xml b/flink-staging/flink-hbase/pom.xml
index 5dc80d9..0612ffd 100644
--- a/flink-staging/flink-hbase/pom.xml
+++ b/flink-staging/flink-hbase/pom.xml
@@ -59,7 +59,7 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
+			<artifactId>flink-streaming-java</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
deleted file mode 100644
index c42e9b0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
+++ /dev/null
@@ -1,106 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-elasticsearch</artifactId>
-	<name>flink-connector-elasticsearch</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<elasticsearch.version>1.7.1</elasticsearch.version>
-	</properties>
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-        <dependency>
-            <groupId>org.elasticsearch</groupId>
-            <artifactId>elasticsearch</artifactId>
-            <version>${elasticsearch.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-            <version>${guava.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-core</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-tests</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-    </dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<rerunFailingTestsCount>3</rerunFailingTestsCount>
-				</configuration>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-failsafe-plugin</artifactId>
-				<configuration>
-					<rerunFailingTestsCount>3</rerunFailingTestsCount>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
deleted file mode 100644
index 546ec8d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
+++ /dev/null
@@ -1,315 +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.connectors.elasticsearch;
-
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.elasticsearch.action.bulk.BulkItemResponse;
-import org.elasticsearch.action.bulk.BulkProcessor;
-import org.elasticsearch.action.bulk.BulkRequest;
-import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.client.transport.TransportClient;
-import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.common.collect.ImmutableList;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.TransportAddress;
-import org.elasticsearch.common.unit.ByteSizeUnit;
-import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.node.Node;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
-
-
-/**
- * Sink that emits its input elements to an Elasticsearch cluster.
- *
- * <p>
- * When using the first constructor {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)}
- * the sink will create a local {@link Node} for communicating with the
- * Elasticsearch cluster. When using the second constructor
- * {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)} a {@link TransportClient} will
- * be used instead.
- *
- * <p>
- * <b>Attention: </b> When using the {@code TransportClient} the sink will fail if no cluster
- * can be connected to. With the {@code Node Client} the sink will block and wait for a cluster
- * to come online.
- *
- * <p>
- * The {@link Map} passed to the constructor is forwarded to Elasticsearch when creating
- * the {@link Node} or {@link TransportClient}. The config keys can be found in the Elasticsearch
- * documentation. An important setting is {@code cluster.name}, this should be set to the name
- * of the cluster that the sink should emit to.
- *
- * <p>
- * Internally, the sink will use a {@link BulkProcessor} to send {@link IndexRequest IndexRequests}.
- * This will buffer elements before sending a request to the cluster. The behaviour of the
- * {@code BulkProcessor} can be configured using these config keys:
- * <ul>
- *   <li> {@code bulk.flush.max.actions}: Maximum amount of elements to buffer
- *   <li> {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer
- *   <li> {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two
- *   settings in milliseconds
- * </ul>
- *
- * <p>
- * You also have to provide an {@link IndexRequestBuilder}. This is used to create an
- * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
- * {@link org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder} for an example.
- *
- * @param <T> Type of the elements emitted by this sink
- */
-public class ElasticsearchSink<T> extends RichSinkFunction<T> {
-
-	public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions";
-	public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb";
-	public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms";
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class);
-
-	/**
-	 * The user specified config map that we forward to Elasticsearch when we create the Client.
-	 */
-	private final Map<String, String> userConfig;
-
-	/**
-	 * The list of nodes that the TransportClient should connect to. This is null if we are using
-	 * an embedded Node to get a Client.
-	 */
-	private final List<TransportAddress> transportNodes;
-
-	/**
-	 * The builder that is used to construct an {@link IndexRequest} from the incoming element.
-	 */
-	private final IndexRequestBuilder<T> indexRequestBuilder;
-
-	/**
-	 * The embedded Node that is used to communicate with the Elasticsearch cluster. This is null
-	 * if we are using a TransportClient.
-	 */
-	private transient Node node;
-
-	/**
-	 * The Client that was either retrieved from a Node or is a TransportClient.
-	 */
-	private transient Client client;
-
-	/**
-	 * Bulk processor that was created using the client
-	 */
-	private transient BulkProcessor bulkProcessor;
-
-	/**
-	 * This is set from inside the BulkProcessor listener if there where failures in processing.
-	 */
-	private final AtomicBoolean hasFailure = new AtomicBoolean(false);
-
-	/**
-	 * This is set from inside the BulkProcessor listener if a Throwable was thrown during processing.
-	 */
-	private final AtomicReference<Throwable> failureThrowable = new AtomicReference<>();
-
-	/**
-	 * Creates a new ElasticsearchSink that connects to the cluster using an embedded Node.
-	 *
-	 * @param userConfig The map of user settings that are passed when constructing the Node and BulkProcessor
-	 * @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
-	 */
-	public ElasticsearchSink(Map<String, String> userConfig, IndexRequestBuilder<T> indexRequestBuilder) {
-		this.userConfig = userConfig;
-		this.indexRequestBuilder = indexRequestBuilder;
-		transportNodes = null;
-	}
-
-	/**
-	 * Creates a new ElasticsearchSink that connects to the cluster using a TransportClient.
-	 *
-	 * @param userConfig The map of user settings that are passed when constructing the TransportClient and BulkProcessor
-	 * @param transportNodes The Elasticsearch Nodes to which to connect using a {@code TransportClient}
-	 * @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
-	 *
-	 */
-	public ElasticsearchSink(Map<String, String> userConfig, List<TransportAddress> transportNodes, IndexRequestBuilder<T> indexRequestBuilder) {
-		this.userConfig = userConfig;
-		this.indexRequestBuilder = indexRequestBuilder;
-		this.transportNodes = transportNodes;
-	}
-
-	/**
-	 * Initializes the connection to Elasticsearch by either creating an embedded
-	 * {@link org.elasticsearch.node.Node} and retrieving the
-	 * {@link org.elasticsearch.client.Client} from it or by creating a
-	 * {@link org.elasticsearch.client.transport.TransportClient}.
-	 */
-	@Override
-	public void open(Configuration configuration) {
-		if (transportNodes == null) {
-			// Make sure that we disable http access to our embedded node
-			Settings settings =
-					ImmutableSettings.settingsBuilder()
-							.put(userConfig)
-							.put("http.enabled", false)
-							.build();
-
-			node =
-					nodeBuilder()
-							.settings(settings)
-							.client(true)
-							.data(false)
-							.node();
-
-			client = node.client();
-
-			if (LOG.isInfoEnabled()) {
-				LOG.info("Created Elasticsearch Client {} from embedded Node", client);
-			}
-
-		} else {
-			Settings settings = ImmutableSettings.settingsBuilder()
-					.put(userConfig)
-					.build();
-
-			TransportClient transportClient = new TransportClient(settings);
-			for (TransportAddress transport: transportNodes) {
-				transportClient.addTransportAddress(transport);
-			}
-
-			// verify that we actually are connected to a cluster
-			ImmutableList<DiscoveryNode> nodes = transportClient.connectedNodes();
-			if (nodes.isEmpty()) {
-				throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
-			} else {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Connected to nodes: " + nodes.toString());
-				}
-			}
-
-			client = transportClient;
-
-			if (LOG.isInfoEnabled()) {
-				LOG.info("Created Elasticsearch TransportClient {}", client);
-			}
-		}
-
-		BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder(
-				client,
-				new BulkProcessor.Listener() {
-					@Override
-					public void beforeBulk(long executionId,
-							BulkRequest request) {
-
-					}
-
-					@Override
-					public void afterBulk(long executionId,
-							BulkRequest request,
-							BulkResponse response) {
-						if (response.hasFailures()) {
-							for (BulkItemResponse itemResp : response.getItems()) {
-								if (itemResp.isFailed()) {
-									LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage());
-									failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage()));
-								}
-							}
-							hasFailure.set(true);
-						}
-					}
-
-					@Override
-					public void afterBulk(long executionId,
-							BulkRequest request,
-							Throwable failure) {
-						LOG.error(failure.getMessage());
-						failureThrowable.compareAndSet(null, failure);
-						hasFailure.set(true);
-					}
-				});
-
-		// This makes flush() blocking
-		bulkProcessorBuilder.setConcurrentRequests(0);
-
-		ParameterTool params = ParameterTool.fromMap(userConfig);
-
-		if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) {
-			bulkProcessorBuilder.setBulkActions(params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS));
-		}
-
-		if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) {
-			bulkProcessorBuilder.setBulkSize(new ByteSizeValue(params.getInt(
-					CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB), ByteSizeUnit.MB));
-		}
-
-		if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) {
-			bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)));
-		}
-
-		bulkProcessor = bulkProcessorBuilder.build();
-	}
-
-	@Override
-	public void invoke(T element) {
-		IndexRequest indexRequest = indexRequestBuilder.createIndexRequest(element, getRuntimeContext());
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Emitting IndexRequest: {}", indexRequest);
-		}
-
-		bulkProcessor.add(indexRequest);
-	}
-
-	@Override
-	public void close() {
-		if (bulkProcessor != null) {
-			bulkProcessor.close();
-			bulkProcessor = null;
-		}
-
-		if (client != null) {
-			client.close();
-		}
-
-		if (node != null) {
-			node.close();
-		}
-
-		if (hasFailure.get()) {
-			Throwable cause = failureThrowable.get();
-			if (cause != null) {
-				throw new RuntimeException("An error occured in ElasticsearchSink.", cause);
-			} else {
-				throw new RuntimeException("An error occured in ElasticsearchSink.");
-
-			}
-		}
-	}
-
-}


[21/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index b94e530..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ /dev/null
@@ -1,470 +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.ExecutionConfig;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-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.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.time.Time;
-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.windowing.triggers.ContinuousEventTimeTrigger;
-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.EventTimeTrigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-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.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-@RunWith(Parameterized.class)
-public class WindowOperatorTest {
-
-	@SuppressWarnings("unchecked,rawtypes")
-	private WindowBufferFactory windowBufferFactory;
-
-	public WindowOperatorTest(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 = 3;
-		final int WINDOW_SLIDE = 1;
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				SlidingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				windowBufferFactory,
-				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
-				EventTimeTrigger.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<>("key1", 1), initialTime + 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime + 999));
-
-		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 StreamRecord<>(new Tuple2<>("key1", 3), 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<>("key1", 3), 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<>("key1", 3), 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 = 3;
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>(
-				TumblingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				windowBufferFactory,
-				new ReduceWindowFunction<String, TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
-				EventTimeTrigger.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<>("key1", 1), initialTime + 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime + 999));
-
-		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<>("key1", 3), 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 = 3;
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new WindowOperator<>(
-				GlobalWindows.create(),
-				new GlobalWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				windowBufferFactory,
-				new ReduceWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
-				ContinuousEventTimeTrigger.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)));
-
-		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...
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime));
-
-		// 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<>("key1", 1), initialTime + 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime + 999));
-
-		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 StreamRecord<>(new Tuple2<>("key1", 3), Long.MAX_VALUE));
-		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<>("key1", 3), Long.MAX_VALUE));
-		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;
-
-		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new WindowOperator<>(
-				GlobalWindows.create(),
-				new GlobalWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				windowBufferFactory,
-				new ReduceWindowFunction<String, 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<>("key1", 1), initialTime + 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime + 999));
-
-		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<>("key1", 1), initialTime + 10999));
-
-		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<>("key1", 4), 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.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/06f6ac5d/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
deleted file mode 100644
index 13766a1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
+++ /dev/null
@@ -1,265 +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.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.WindowedStream;
-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.SlidingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-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.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
-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;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * These tests verify that the api calls on
- * {@link WindowedStream} 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));
-		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduce(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(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(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);
-	}
-
-	/**
-	 * These tests ensure that the correct trigger is set when using event-time windows.
-	 */
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testEventTime() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.keyBy(0)
-				.window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduce(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.assertFalse(winOperator1.isSetProcessingTime());
-		Assert.assertTrue(winOperator1.getTrigger() instanceof EventTimeTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(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.assertFalse(winOperator2.isSetProcessingTime());
-		Assert.assertTrue(winOperator2.getTrigger() instanceof EventTimeTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testNonEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		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(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.trigger(CountTrigger.of(100))
-				.reduce(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.isSetProcessingTime());
-		Assert.assertTrue(winOperator1.getTrigger() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-				.trigger(CountTrigger.of(100))
-				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(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.isSetProcessingTime());
-		Assert.assertTrue(winOperator2.getTrigger() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
-
-		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(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.evictor(CountEvictor.of(100))
-				.reduce(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.assertFalse(winOperator1.isSetProcessingTime());
-		Assert.assertTrue(winOperator1.getTrigger() instanceof EventTimeTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
-		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.keyBy(0)
-				.window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-				.trigger(CountTrigger.of(100))
-				.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
-				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(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.assertFalse(winOperator2.isSetProcessingTime());
-		Assert.assertTrue(winOperator2.getTrigger() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
-		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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java
deleted file mode 100644
index a1cea13..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitionerTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.assertArrayEquals;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class BroadcastPartitionerTest {
-
-	private BroadcastPartitioner<Tuple> broadcastPartitioner1;
-	private BroadcastPartitioner<Tuple> broadcastPartitioner2;
-	private BroadcastPartitioner<Tuple> broadcastPartitioner3;
-	
-	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
-	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(null);
-
-	@Before
-	public void setPartitioner() {
-		broadcastPartitioner1 = new BroadcastPartitioner<Tuple>();
-		broadcastPartitioner2 = new BroadcastPartitioner<Tuple>();
-		broadcastPartitioner3 = new BroadcastPartitioner<Tuple>();
-
-	}
-
-	@Test
-	public void testSelectChannels() {
-		int[] first = new int[] { 0 };
-		int[] second = new int[] { 0, 1 };
-		int[] sixth = new int[] { 0, 1, 2, 3, 4, 5 };
-		sd.setInstance(streamRecord);
-		assertArrayEquals(first, broadcastPartitioner1.selectChannels(sd, 1));
-		assertArrayEquals(second, broadcastPartitioner2.selectChannels(sd, 2));
-		assertArrayEquals(sixth, broadcastPartitioner3.selectChannels(sd, 6));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java
deleted file mode 100644
index f7bd739..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitionerTest.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class ForwardPartitionerTest {
-
-	private ForwardPartitioner<Tuple> forwardPartitioner;
-	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
-	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
-			null);
-
-	@Before
-	public void setPartitioner() {
-		forwardPartitioner = new ForwardPartitioner<Tuple>();
-	}
-
-	@Test
-	public void testSelectChannelsLength() {
-		sd.setInstance(streamRecord);
-		assertEquals(1, forwardPartitioner.selectChannels(sd, 1).length);
-		assertEquals(1, forwardPartitioner.selectChannels(sd, 2).length);
-		assertEquals(1, forwardPartitioner.selectChannels(sd, 1024).length);
-	}
-
-	@Test
-	public void testSelectChannelsInterval() {
-		sd.setInstance(streamRecord);
-		assertEquals(0, forwardPartitioner.selectChannels(sd, 1)[0]);
-		assertEquals(0, forwardPartitioner.selectChannels(sd, 2)[0]);
-		assertEquals(0, forwardPartitioner.selectChannels(sd, 1024)[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java
deleted file mode 100644
index 6ae3730..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitionerTest.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.assertArrayEquals;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class GlobalPartitionerTest {
-
-	private GlobalPartitioner<Tuple> globalPartitioner;
-	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
-	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
-			null);
-
-	@Before
-	public void setPartitioner() {
-		globalPartitioner = new GlobalPartitioner<Tuple>();
-	}
-
-	@Test
-	public void testSelectChannels() {
-		int[] result = new int[] { 0 };
-
-		sd.setInstance(streamRecord);
-
-		assertArrayEquals(result, globalPartitioner.selectChannels(sd, 1));
-		assertArrayEquals(result, globalPartitioner.selectChannels(sd, 2));
-		assertArrayEquals(result, globalPartitioner.selectChannels(sd, 1024));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java
deleted file mode 100644
index 6dbf932..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/HashPartitionerTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class HashPartitionerTest {
-
-	private HashPartitioner<Tuple2<String, Integer>> hashPartitioner;
-	private StreamRecord<Tuple2<String, Integer>> streamRecord1 = new StreamRecord<Tuple2<String, Integer>>(new Tuple2<String, Integer>("test", 0));
-	private StreamRecord<Tuple2<String, Integer>> streamRecord2 = new StreamRecord<Tuple2<String, Integer>>(new Tuple2<String, Integer>("test", 42));
-	private SerializationDelegate<StreamRecord<Tuple2<String, Integer>>> sd1 = new SerializationDelegate<StreamRecord<Tuple2<String, Integer>>>(null);
-	private SerializationDelegate<StreamRecord<Tuple2<String, Integer>>> sd2 = new SerializationDelegate<StreamRecord<Tuple2<String, Integer>>>(null);
-
-	@Before
-	public void setPartitioner() {
-		hashPartitioner = new HashPartitioner<Tuple2<String, Integer>>(new KeySelector<Tuple2<String, Integer>, String>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public String getKey(Tuple2<String, Integer> value) throws Exception {
-				return value.getField(0);
-			}
-		});
-	}
-
-	@Test
-	public void testSelectChannelsLength() {
-		sd1.setInstance(streamRecord1);
-		assertEquals(1, hashPartitioner.selectChannels(sd1, 1).length);
-		assertEquals(1, hashPartitioner.selectChannels(sd1, 2).length);
-		assertEquals(1, hashPartitioner.selectChannels(sd1, 1024).length);
-	}
-
-	@Test
-	public void testSelectChannelsGrouping() {
-		sd1.setInstance(streamRecord1);
-		sd2.setInstance(streamRecord2);
-
-		assertArrayEquals(hashPartitioner.selectChannels(sd1, 1),
-				hashPartitioner.selectChannels(sd2, 1));
-		assertArrayEquals(hashPartitioner.selectChannels(sd1, 2),
-				hashPartitioner.selectChannels(sd2, 2));
-		assertArrayEquals(hashPartitioner.selectChannels(sd1, 1024),
-				hashPartitioner.selectChannels(sd2, 1024));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
deleted file mode 100644
index aa70e8a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitionerTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class RebalancePartitionerTest {
-	
-	private RebalancePartitioner<Tuple> distributePartitioner;
-	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
-	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
-			null);
-	
-	@Before
-	public void setPartitioner() {
-		distributePartitioner = new RebalancePartitioner<Tuple>();
-	}
-	
-	@Test
-	public void testSelectChannelsLength() {
-		sd.setInstance(streamRecord);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 1).length);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 2).length);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 1024).length);
-	}
-	
-	@Test
-	public void testSelectChannelsInterval() {
-		sd.setInstance(streamRecord);
-		assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
-		assertEquals(1, distributePartitioner.selectChannels(sd, 3)[0]);
-		assertEquals(2, distributePartitioner.selectChannels(sd, 3)[0]);
-		assertEquals(0, distributePartitioner.selectChannels(sd, 3)[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java
deleted file mode 100644
index aff177c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitionerTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Before;
-import org.junit.Test;
-
-public class ShufflePartitionerTest {
-
-	private ShufflePartitioner<Tuple> shufflePartitioner;
-	private StreamRecord<Tuple> streamRecord = new StreamRecord<Tuple>(null);
-	private SerializationDelegate<StreamRecord<Tuple>> sd = new SerializationDelegate<StreamRecord<Tuple>>(
-			null);
-
-	@Before
-	public void setPartitioner() {
-		shufflePartitioner = new ShufflePartitioner<Tuple>();
-	}
-
-	@Test
-	public void testSelectChannelsLength() {
-		sd.setInstance(streamRecord);
-		assertEquals(1, shufflePartitioner.selectChannels(sd, 1).length);
-		assertEquals(1, shufflePartitioner.selectChannels(sd, 2).length);
-		assertEquals(1, shufflePartitioner.selectChannels(sd, 1024).length);
-	}
-
-	@Test
-	public void testSelectChannelsInterval() {
-		sd.setInstance(streamRecord);
-		assertEquals(0, shufflePartitioner.selectChannels(sd, 1)[0]);
-
-		assertTrue(0 <= shufflePartitioner.selectChannels(sd, 2)[0]);
-		assertTrue(2 > shufflePartitioner.selectChannels(sd, 2)[0]);
-
-		assertTrue(0 <= shufflePartitioner.selectChannels(sd, 1024)[0]);
-		assertTrue(1024 > shufflePartitioner.selectChannels(sd, 1024)[0]);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.java
deleted file mode 100644
index d48f7f4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializerTest.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.runtime.streamrecord;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-public class StreamRecordSerializerTest {
-	
-	@Test
-	public void testDeepDuplication() {
-		try {
-			@SuppressWarnings("unchecked")
-			TypeSerializer<Long> serializer1 = (TypeSerializer<Long>) mock(TypeSerializer.class);
-			@SuppressWarnings("unchecked")
-			TypeSerializer<Long> serializer2 = (TypeSerializer<Long>) mock(TypeSerializer.class);
-			
-			when(serializer1.duplicate()).thenReturn(serializer2);
-			
-			StreamRecordSerializer<Long> streamRecSer = new StreamRecordSerializer<Long>(serializer1);
-			assertEquals(serializer1, streamRecSer.getContainedTypeSerializer());
-			
-			StreamRecordSerializer<Long> copy = streamRecSer.duplicate();
-			assertNotEquals(copy, streamRecSer);
-			assertNotEquals(copy.getContainedTypeSerializer(), streamRecSer.getContainedTypeSerializer());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testBasicProperties() {
-		try {
-			StreamRecordSerializer<Long> streamRecSer = new StreamRecordSerializer<Long>(LongSerializer.INSTANCE);
-			
-			assertFalse(streamRecSer.isImmutableType());
-			assertEquals(Long.class, streamRecSer.createInstance().getValue().getClass());
-			assertEquals(LongSerializer.INSTANCE.getLength(), streamRecSer.getLength());
-			
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java
deleted file mode 100644
index 4c6957b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java
+++ /dev/null
@@ -1,334 +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.tasks;
-
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-/**
- * Tests for {@link OneInputStreamTask}.
- *
- * <p>
- * Note:<br>
- * We only use a {@link StreamMap} operator here. We also test the individual operators but Map is
- * used as a representative to test OneInputStreamTask, since OneInputStreamTask is used for all
- * OneInputStreamOperators.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ResultPartitionWriter.class})
-public class OneInputStreamTaskTest {
-
-	/**
-	 * This test verifies that open() and close() are correctly called. This test also verifies
-	 * that timestamps of emitted elements are correct. {@link StreamMap} assigns the input
-	 * timestamp to emitted elements.
-	 */
-	@Test
-	public void testOpenCloseAndTimestamps() throws Exception {
-		final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<String, String>();
-		final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<String, String>(mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		StreamMap<String, String> mapOperator = new StreamMap<String, String>(new TestOpenCloseMapFunction());
-		streamConfig.setStreamOperator(mapOperator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.invoke();
-
-		testHarness.processElement(new StreamRecord<String>("Hello", initialTime + 1));
-		testHarness.processElement(new StreamRecord<String>("Ciao", initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("Hello", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("Ciao", initialTime + 2));
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseMapFunction.closeCalled);
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				expectedOutput,
-				testHarness.getOutput());
-	}
-
-	/**
-	 * This test verifies that watermarks are correctly forwarded. This also checks whether
-	 * watermarks are forwarded only when we have received watermarks from all inputs. The
-	 * forwarded watermark must be the minimum of the watermarks of all inputs.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testWatermarkForwarding() throws Exception {
-		final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<String, String>();
-		final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<String, String>(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		StreamMap<String, String> mapOperator = new StreamMap<String, String>(new IdentityMap());
-		streamConfig.setStreamOperator(mapOperator);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-		long initialTime = 0L;
-
-		testHarness.invoke();
-
-		testHarness.processElement(new Watermark(initialTime), 0, 0);
-		testHarness.processElement(new Watermark(initialTime), 0, 1);
-		testHarness.processElement(new Watermark(initialTime), 1, 0);
-
-		// now the output should still be empty
-		testHarness.waitForInputProcessing();
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		testHarness.processElement(new Watermark(initialTime), 1, 1);
-
-		// now the watermark should have propagated, Map simply forward Watermarks
-		testHarness.waitForInputProcessing();
-		expectedOutput.add(new Watermark(initialTime));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				expectedOutput,
-				testHarness.getOutput());
-
-		// contrary to checkpoint barriers these elements are not blocked by watermarks
-		testHarness.processElement(new StreamRecord<String>("Hello", initialTime));
-		testHarness.processElement(new StreamRecord<String>("Ciao", initialTime));
-		expectedOutput.add(new StreamRecord<String>("Hello", initialTime));
-		expectedOutput.add(new StreamRecord<String>("Ciao", initialTime));
-
-		testHarness.processElement(new Watermark(initialTime + 4), 0, 0);
-		testHarness.processElement(new Watermark(initialTime + 3), 0, 1);
-		testHarness.processElement(new Watermark(initialTime + 3), 1, 0);
-		testHarness.processElement(new Watermark(initialTime + 2), 1, 1);
-
-		// check whether we get the minimum of all the watermarks, this must also only occur in
-		// the output after the two StreamRecords
-		testHarness.waitForInputProcessing();
-		expectedOutput.add(new Watermark(initialTime + 2));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-
-		// advance watermark from one of the inputs, now we should get a now one since the
-		// minimum increases
-		testHarness.processElement(new Watermark(initialTime + 4), 1, 1);
-		testHarness.waitForInputProcessing();
-		expectedOutput.add(new Watermark(initialTime + 3));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		// advance the other two inputs, now we should get a new one since the
-		// minimum increases again
-		testHarness.processElement(new Watermark(initialTime + 4), 0, 1);
-		testHarness.processElement(new Watermark(initialTime + 4), 1, 0);
-		testHarness.waitForInputProcessing();
-		expectedOutput.add(new Watermark(initialTime + 4));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		List<String> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput());
-		Assert.assertEquals(2, resultElements.size());
-	}
-
-	/**
-	 * This test verifies that checkpoint barriers are correctly forwarded.
-	 */
-	@Test
-	public void testCheckpointBarriers() throws Exception {
-		final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<String, String>();
-		final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<String, String>(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		StreamMap<String, String> mapOperator = new StreamMap<String, String>(new IdentityMap());
-		streamConfig.setStreamOperator(mapOperator);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-		long initialTime = 0L;
-
-		testHarness.invoke();
-
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0);
-
-		// These elements should be buffered until we receive barriers from
-		// all inputs
-		testHarness.processElement(new StreamRecord<String>("Hello-0-0", initialTime), 0, 0);
-		testHarness.processElement(new StreamRecord<String>("Ciao-0-0", initialTime), 0, 0);
-
-		// These elements should be forwarded, since we did not yet receive a checkpoint barrier
-		// on that input, only add to same input, otherwise we would not know the ordering
-		// of the output since the Task might read the inputs in any order
-		testHarness.processElement(new StreamRecord<String>("Hello-1-1", initialTime), 1, 1);
-		testHarness.processElement(new StreamRecord<String>("Ciao-1-1", initialTime), 1, 1);
-		expectedOutput.add(new StreamRecord<String>("Hello-1-1", initialTime));
-		expectedOutput.add(new StreamRecord<String>("Ciao-1-1", initialTime));
-
-		testHarness.waitForInputProcessing();
-		// we should not yet see the barrier, only the two elements from non-blocked input
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1);
-
-		testHarness.waitForInputProcessing();
-
-		// now we should see the barrier and after that the buffered elements
-		expectedOutput.add(new CheckpointBarrier(0, 0));
-		expectedOutput.add(new StreamRecord<String>("Hello-0-0", initialTime));
-		expectedOutput.add(new StreamRecord<String>("Ciao-0-0", initialTime));
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	/**
-	 * This test verifies that checkpoint barriers and barrier buffers work correctly with
-	 * concurrent checkpoint barriers where one checkpoint is "overtaking" another checkpoint, i.e.
-	 * some inputs receive barriers from an earlier checkpoint, thereby blocking,
-	 * then all inputs receive barriers from a later checkpoint.
-	 */
-	@Test
-	public void testOvertakingCheckpointBarriers() throws Exception {
-		final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<String, String>();
-		final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<String, String>(mapTask, 2, 2, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		StreamMap<String, String> mapOperator = new StreamMap<String, String>(new IdentityMap());
-		streamConfig.setStreamOperator(mapOperator);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-		long initialTime = 0L;
-
-		testHarness.invoke();
-
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0);
-
-		// These elements should be buffered until we receive barriers from
-		// all inputs
-		testHarness.processElement(new StreamRecord<String>("Hello-0-0", initialTime), 0, 0);
-		testHarness.processElement(new StreamRecord<String>("Ciao-0-0", initialTime), 0, 0);
-
-		// These elements should be forwarded, since we did not yet receive a checkpoint barrier
-		// on that input, only add to same input, otherwise we would not know the ordering
-		// of the output since the Task might read the inputs in any order
-		testHarness.processElement(new StreamRecord<String>("Hello-1-1", initialTime), 1, 1);
-		testHarness.processElement(new StreamRecord<String>("Ciao-1-1", initialTime), 1, 1);
-		expectedOutput.add(new StreamRecord<String>("Hello-1-1", initialTime));
-		expectedOutput.add(new StreamRecord<String>("Ciao-1-1", initialTime));
-
-		testHarness.waitForInputProcessing();
-		// we should not yet see the barrier, only the two elements from non-blocked input
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		// Now give a later barrier to all inputs, this should unblock the first channel,
-		// thereby allowing the two blocked elements through
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 0);
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 1);
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 0);
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 1);
-
-		expectedOutput.add(new StreamRecord<String>("Hello-0-0", initialTime));
-		expectedOutput.add(new StreamRecord<String>("Ciao-0-0", initialTime));
-		expectedOutput.add(new CheckpointBarrier(1, 1));
-
-		testHarness.waitForInputProcessing();
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-
-		// Then give the earlier barrier, these should be ignored
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1);
-
-		testHarness.waitForInputProcessing();
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseMapFunction extends RichMapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public String map(String value) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return value;
-		}
-	}
-
-	private static class IdentityMap implements MapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map(String value) throws Exception {
-			return value;
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
deleted file mode 100644
index 7fb8ba3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.tasks;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate;
-
-import java.io.IOException;
-
-
-/**
- * Test harness for testing a {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}.
- *
- * <p>
- * This mock Invokable provides the task with a basic runtime context and allows pushing elements
- * and watermarks into the task. {@link #getOutput()} can be used to get the emitted elements
- * and events. You are free to modify the retrieved list.
- *
- * <p>
- * After setting up everything the Task can be invoked using {@link #invoke()}. This will start
- * a new Thread to execute the Task. Use {@link #waitForTaskCompletion()} to wait for the Task
- * thread to finish. Use {@link #processElement} to send elements to the task. Use
- * {@link #processEvent(AbstractEvent)} to send events to the task.
- * Before waiting for the task to finish you must call {@link #endInput()} to signal to the task
- * that data entry is finished.
- *
- * <p>
- * When Elements or Events are offered to the Task they are put into a queue. The input gates
- * of the Task read from this queue. Use {@link #waitForInputProcessing()} to wait until all
- * queues are empty. This must be used after entering some elements before checking the
- * desired output.
- *
- * <p>
- * When using this you need to add the following line to your test class to setup Powermock:
- * {@code @PrepareForTest({ResultPartitionWriter.class})}
- */
-public class OneInputStreamTaskTestHarness<IN, OUT> extends StreamTaskTestHarness<OUT> {
-
-	private TypeInformation<IN> inputType;
-	private TypeSerializer<IN> inputSerializer;
-
-	/**
-	 * Creates a test harness with the specified number of input gates and specified number
-	 * of channels per input gate.
-	 */
-	public OneInputStreamTaskTestHarness(OneInputStreamTask<IN, OUT> task,
-			int numInputGates,
-			int numInputChannelsPerGate,
-			TypeInformation<IN> inputType,
-			TypeInformation<OUT> outputType) {
-		super(task, outputType);
-
-		this.inputType = inputType;
-		inputSerializer = inputType.createSerializer(executionConfig);
-
-		this.numInputGates = numInputGates;
-		this.numInputChannelsPerGate = numInputChannelsPerGate;
-	}
-
-	/**
-	 * Creates a test harness with one input gate that has one input channel.
-	 */
-	public OneInputStreamTaskTestHarness(OneInputStreamTask<IN, OUT> task,
-			TypeInformation<IN> inputType,
-			TypeInformation<OUT> outputType) {
-		this(task, 1, 1, inputType, outputType);
-	}
-
-	@Override
-	protected void initializeInputs() throws IOException, InterruptedException {
-		inputGates = new StreamTestSingleInputGate[numInputGates];
-
-		for (int i = 0; i < numInputGates; i++) {
-			inputGates[i] = new StreamTestSingleInputGate<IN>(
-					numInputChannelsPerGate,
-					bufferSize,
-					inputSerializer);
-			this.mockEnv.addInputGate(inputGates[i].getInputGate());
-		}
-
-
-		streamConfig.setNumberOfInputs(1);
-		streamConfig.setTypeSerializerIn1(inputSerializer);
-	}
-
-}
-


[31/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 5de6cd1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
+++ /dev/null
@@ -1,539 +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 com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
-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.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Window operator for non-keyed windows.
- *
- * @see org.apache.flink.streaming.runtime.operators.windowing.WindowOperator
- *
- * @param <IN> The type of the incoming elements.
- * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
- * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
- */
-public class NonKeyedWindowOperator<IN, OUT, W extends Window>
-		extends AbstractUdfStreamOperator<OUT, AllWindowFunction<IN, OUT, W>>
-		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable, OutputTypeConfigurable<OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(WindowOperator.class);
-
-	// ------------------------------------------------------------------------
-	// Configuration values and stuff from the user
-	// ------------------------------------------------------------------------
-
-	private final WindowAssigner<? super IN, W> windowAssigner;
-
-	private final Trigger<? super IN, ? super W> trigger;
-
-	private final WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory;
-
-	/**
-	 * If this is true. The current processing time is set as the timestamp of incoming elements.
-	 * This for use with a {@link org.apache.flink.streaming.api.windowing.evictors.TimeEvictor}
-	 * if eviction should happen based on processing time.
-	 */
-	private boolean setProcessingTime = false;
-
-	/**
-	 * This is used to copy the incoming element because it can be put into several window
-	 * buffers.
-	 */
-	private TypeSerializer<IN> inputSerializer;
-
-	/**
-	 * For serializing the window in checkpoints.
-	 */
-	private final TypeSerializer<W> windowSerializer;
-
-	// ------------------------------------------------------------------------
-	// State that is not checkpointed
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Processing time timers that are currently in-flight.
-	 */
-	private transient Map<Long, Set<Context>> processingTimeTimers;
-
-	/**
-	 * Current waiting watermark callbacks.
-	 */
-	private transient Map<Long, Set<Context>> watermarkTimers;
-
-	/**
-	 * This is given to the {@code WindowFunction} for emitting elements with a given timestamp.
-	 */
-	protected transient TimestampedCollector<OUT> timestampedCollector;
-
-	// ------------------------------------------------------------------------
-	// State that needs to be checkpointed
-	// ------------------------------------------------------------------------
-
-	/**
-	 * The windows (panes) that are currently in-flight. Each pane has a {@code WindowBuffer}
-	 * and a {@code TriggerContext} that stores the {@code Trigger} for that pane.
-	 */
-	protected transient Map<W, Context> windows;
-
-	/**
-	 * Creates a new {@code WindowOperator} based on the given policies and user functions.
-	 */
-	public NonKeyedWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
-			TypeSerializer<W> windowSerializer,
-			WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory,
-			AllWindowFunction<IN, OUT, W> windowFunction,
-			Trigger<? super IN, ? super W> trigger) {
-
-		super(windowFunction);
-
-		this.windowAssigner = requireNonNull(windowAssigner);
-		this.windowSerializer = windowSerializer;
-
-		this.windowBufferFactory = requireNonNull(windowBufferFactory);
-		this.trigger = requireNonNull(trigger);
-
-		setChainingStrategy(ChainingStrategy.ALWAYS);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public final void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
-		inputSerializer = (TypeSerializer<IN>) type.createSerializer(executionConfig);
-	}
-
-	@Override
-	public final void open() throws Exception {
-		super.open();
-		timestampedCollector = new TimestampedCollector<>(output);
-
-		if (inputSerializer == null) {
-			throw new IllegalStateException("Input serializer was not set.");
-		}
-
-		windowBufferFactory.setRuntimeContext(getRuntimeContext());
-		windowBufferFactory.open(getUserFunctionParameters());
-
-		// these could already be initialized from restoreState()
-		if (watermarkTimers == null) {
-			watermarkTimers = new HashMap<>();
-		}
-		if (processingTimeTimers == null) {
-			processingTimeTimers = new HashMap<>();
-		}
-		if (windows == null) {
-			windows = new HashMap<>();
-		}
-
-		// re-register timers that this window context had set
-		for (Context context: windows.values()) {
-			if (context.processingTimeTimer > 0) {
-				Set<Context> triggers = processingTimeTimers.get(context.processingTimeTimer);
-				if (triggers == null) {
-					getRuntimeContext().registerTimer(context.processingTimeTimer, NonKeyedWindowOperator.this);
-					triggers = new HashSet<>();
-					processingTimeTimers.put(context.processingTimeTimer, triggers);
-				}
-				triggers.add(context);
-			}
-			if (context.watermarkTimer > 0) {
-				Set<Context> triggers = watermarkTimers.get(context.watermarkTimer);
-				if (triggers == null) {
-					triggers = new HashSet<>();
-					watermarkTimers.put(context.watermarkTimer, triggers);
-				}
-				triggers.add(context);
-			}
-
-		}
-	}
-
-	@Override
-	public final void close() throws Exception {
-		super.close();
-		// emit the elements that we still keep
-		for (Context window: windows.values()) {
-			emitWindow(window);
-		}
-		windows.clear();
-		windowBufferFactory.close();
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public final 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) {
-			Context context = windows.get(window);
-			if (context == null) {
-				WindowBuffer<IN> windowBuffer = windowBufferFactory.create();
-				context = new Context(window, windowBuffer);
-				windows.put(window, context);
-			}
-			StreamRecord<IN> elementCopy = new StreamRecord<>(inputSerializer.copy(element.getValue()), element.getTimestamp());
-			context.windowBuffer.storeElement(elementCopy);
-			Trigger.TriggerResult triggerResult = trigger.onElement(elementCopy.getValue(), elementCopy.getTimestamp(), window, context);
-			processTriggerResult(triggerResult, window);
-		}
-	}
-
-	protected void emitWindow(Context context) throws Exception {
-		timestampedCollector.setTimestamp(context.window.maxTimestamp());
-
-		userFunction.apply(
-				context.window,
-				context.windowBuffer.getUnpackedElements(),
-				timestampedCollector);
-	}
-
-	private void processTriggerResult(Trigger.TriggerResult triggerResult, W window) throws Exception {
-		switch (triggerResult) {
-			case FIRE: {
-				Context context = windows.get(window);
-				if (context == null) {
-					LOG.debug("Window {} already gone.", window);
-					return;
-				}
-
-
-				emitWindow(context);
-				break;
-			}
-
-			case FIRE_AND_PURGE: {
-				Context context = windows.remove(window);
-				if (context == null) {
-					LOG.debug("Window {} already gone.", window);
-					return;
-				}
-
-				emitWindow(context);
-				break;
-			}
-
-			case CONTINUE:
-				// ingore
-		}
-	}
-
-	@Override
-	public final void processWatermark(Watermark mark) throws Exception {
-		Set<Long> toRemove = new HashSet<>();
-
-		for (Map.Entry<Long, Set<Context>> triggers: watermarkTimers.entrySet()) {
-			if (triggers.getKey() <= mark.getTimestamp()) {
-				for (Context context: triggers.getValue()) {
-					Trigger.TriggerResult triggerResult = context.onEventTime(triggers.getKey());
-					processTriggerResult(triggerResult, context.window);
-				}
-				toRemove.add(triggers.getKey());
-			}
-		}
-
-		for (Long l: toRemove) {
-			watermarkTimers.remove(l);
-		}
-		output.emitWatermark(mark);
-	}
-
-	@Override
-	public final void trigger(long time) throws Exception {
-		Set<Long> toRemove = new HashSet<>();
-
-		for (Map.Entry<Long, Set<Context>> triggers: processingTimeTimers.entrySet()) {
-			if (triggers.getKey() < time) {
-				for (Context context: triggers.getValue()) {
-					Trigger.TriggerResult triggerResult = context.onProcessingTime(time);
-					processTriggerResult(triggerResult, context.window);
-				}
-				toRemove.add(triggers.getKey());
-			}
-		}
-
-		for (Long l: toRemove) {
-			processingTimeTimers.remove(l);
-		}
-	}
-
-	/**
-	 * A context object that is given to {@code Trigger} functions to allow them to register
-	 * timer/watermark callbacks.
-	 */
-	protected class Context implements Trigger.TriggerContext {
-		protected W window;
-
-		protected WindowBuffer<IN> windowBuffer;
-
-		protected HashMap<String, Serializable> state;
-
-		// use these to only allow one timer in flight at a time of each type
-		// if the trigger registers another timer this value here will be overwritten,
-		// the timer is not removed from the set of in-flight timers to improve performance.
-		// When a trigger fires it is just checked against the last timer that was set.
-		protected long watermarkTimer;
-		protected long processingTimeTimer;
-
-		public Context(
-				W window,
-				WindowBuffer<IN> windowBuffer) {
-			this.window = window;
-			this.windowBuffer = windowBuffer;
-			state = new HashMap<>();
-
-			this.watermarkTimer = -1;
-			this.processingTimeTimer = -1;
-		}
-
-
-		@SuppressWarnings("unchecked")
-		protected Context(DataInputView in) throws Exception {
-			this.window = windowSerializer.deserialize(in);
-			this.watermarkTimer = in.readLong();
-			this.processingTimeTimer = in.readLong();
-
-			int stateSize = in.readInt();
-			byte[] stateData = new byte[stateSize];
-			in.read(stateData);
-			ByteArrayInputStream bais = new ByteArrayInputStream(stateData);
-			state = (HashMap<String, Serializable>) SerializationUtils.deserialize(bais);
-
-			this.windowBuffer = windowBufferFactory.create();
-			int numElements = in.readInt();
-			MultiplexingStreamRecordSerializer<IN> recordSerializer = new MultiplexingStreamRecordSerializer<>(inputSerializer);
-			for (int i = 0; i < numElements; i++) {
-				windowBuffer.storeElement(recordSerializer.deserialize(in).<IN>asRecord());
-			}
-		}
-
-		protected void writeToState(StateBackend.CheckpointStateOutputView out) throws IOException {
-			windowSerializer.serialize(window, out);
-			out.writeLong(watermarkTimer);
-			out.writeLong(processingTimeTimer);
-
-			ByteArrayOutputStream baos = new ByteArrayOutputStream();
-			SerializationUtils.serialize(state, baos);
-			out.writeInt(baos.size());
-			out.write(baos.toByteArray(), 0, baos.size());
-
-			MultiplexingStreamRecordSerializer<IN> recordSerializer = new MultiplexingStreamRecordSerializer<>(inputSerializer);
-			out.writeInt(windowBuffer.size());
-			for (StreamRecord<IN> element: windowBuffer.getElements()) {
-				recordSerializer.serialize(element, out);
-			}
-		}
-
-		@SuppressWarnings("unchecked")
-		public <S extends Serializable> OperatorState<S> getKeyValueState(final String name, final S defaultState) {
-			return new OperatorState<S>() {
-				@Override
-				public S value() throws IOException {
-					Serializable value = state.get(name);
-					if (value == null) {
-						state.put(name, defaultState);
-						value = defaultState;
-					}
-					return (S) value;
-				}
-
-				@Override
-				public void update(S value) throws IOException {
-					state.put(name, value);
-				}
-			};
-		}
-
-		@Override
-		public void registerProcessingTimeTimer(long time) {
-			if (this.processingTimeTimer == time) {
-				// we already have set a trigger for that time
-				return;
-			}
-			Set<Context> triggers = processingTimeTimers.get(time);
-			if (triggers == null) {
-				getRuntimeContext().registerTimer(time, NonKeyedWindowOperator.this);
-				triggers = new HashSet<>();
-				processingTimeTimers.put(time, triggers);
-			}
-			this.processingTimeTimer = time;
-			triggers.add(this);
-		}
-
-		@Override
-		public void registerEventTimeTimer(long time) {
-			if (watermarkTimer == time) {
-				// we already have set a trigger for that time
-				return;
-			}
-			Set<Context> triggers = watermarkTimers.get(time);
-			if (triggers == null) {
-				triggers = new HashSet<>();
-				watermarkTimers.put(time, triggers);
-			}
-			this.watermarkTimer = time;
-			triggers.add(this);
-		}
-
-		public Trigger.TriggerResult onProcessingTime(long time) throws Exception {
-			if (time == processingTimeTimer) {
-				return trigger.onProcessingTime(time, this);
-			} else {
-				return Trigger.TriggerResult.CONTINUE;
-			}
-		}
-
-		public Trigger.TriggerResult onEventTime(long time) throws Exception {
-			if (time == watermarkTimer) {
-				return trigger.onEventTime(time, this);
-			} else {
-				return Trigger.TriggerResult.CONTINUE;
-			}
-		}
-	}
-
-	/**
-	 * 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;
-	}
-
-	@Override
-	public final void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) {
-		if (userFunction instanceof OutputTypeConfigurable) {
-			@SuppressWarnings("unchecked")
-			OutputTypeConfigurable<OUT> typeConfigurable = (OutputTypeConfigurable<OUT>) userFunction;
-			typeConfigurable.setOutputType(outTypeInfo, executionConfig);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpointing
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
-		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
-
-		// we write the panes with the key/value maps into the stream
-		StateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
-
-		int numWindows = windows.size();
-		out.writeInt(numWindows);
-		for (Context context: windows.values()) {
-			context.writeToState(out);
-		}
-
-		taskState.setOperatorState(out.closeAndGetHandle());
-		return taskState;
-	}
-
-	@Override
-	public void restoreState(StreamTaskState taskState) throws Exception {
-		super.restoreState(taskState);
-
-
-		@SuppressWarnings("unchecked")
-		StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
-		DataInputView in = inputState.getState(getUserCodeClassloader());
-
-		int numWindows = in.readInt();
-		this.windows = new HashMap<>(numWindows);
-		this.processingTimeTimers = new HashMap<>();
-		this.watermarkTimers = new HashMap<>();
-
-		for (int j = 0; j < numWindows; j++) {
-			Context context = new Context(in);
-			windows.put(context.window, context);
-		}
-	}
-
-
-	// ------------------------------------------------------------------------
-	// Getters for testing
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	public boolean isSetProcessingTime() {
-		return setProcessingTime;
-	}
-
-	@VisibleForTesting
-	public Trigger<? super IN, ? super W> getTrigger() {
-		return trigger;
-	}
-
-	@VisibleForTesting
-	public WindowAssigner<? super IN, W> getWindowAssigner() {
-		return windowAssigner;
-	}
-
-	@VisibleForTesting
-	public WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> getWindowBufferFactory() {
-		return windowBufferFactory;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
deleted file mode 100644
index 2491c57..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ /dev/null
@@ -1,625 +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 com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.lang.SerializationUtils;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
-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.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * An operator that implements the logic for windowing based on a {@link WindowAssigner} and
- * {@link Trigger}.
- *
- * <p>
- * When an element arrives it gets assigned a key using a {@link KeySelector} and it get's
- * assigned to zero or more windows using a {@link WindowAssigner}. Based on this the element
- * is put into panes. A pane is the bucket of elements that have the same key and same
- * {@code Window}. An element can be in multiple panes of it was assigned to multiple windows by the
- * {@code WindowAssigner}.
- *
- * <p>
- * Each pane gets its own instance of the provided {@code Trigger}. This trigger determines when
- * the contents of the pane should be processed to emit results. When a trigger fires,
- * the given {@link WindowFunction} is invoked to produce the results that are emitted for
- * the pane to which the {@code Trigger} belongs.
- *
- * <p>
- * This operator also needs a {@link WindowBufferFactory} to create a buffer for storing the
- * elements of each pane.
- *
- * @param <K> The type of key returned by the {@code KeySelector}.
- * @param <IN> The type of the incoming elements.
- * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
- * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
- */
-public class WindowOperator<K, IN, OUT, W extends Window>
-		extends AbstractUdfStreamOperator<OUT, WindowFunction<IN, OUT, K, W>>
-		implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable, OutputTypeConfigurable<OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(WindowOperator.class);
-
-	// ------------------------------------------------------------------------
-	// Configuration values and user functions
-	// ------------------------------------------------------------------------
-
-	private final WindowAssigner<? super IN, W> windowAssigner;
-
-	private final KeySelector<IN, K> keySelector;
-
-	private final Trigger<? super IN, ? super W> trigger;
-
-	private final WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory;
-
-	/**
-	 * If this is true. The current processing time is set as the timestamp of incoming elements.
-	 * This for use with a {@link org.apache.flink.streaming.api.windowing.evictors.TimeEvictor}
-	 * if eviction should happen based on processing time.
-	 */
-	private boolean setProcessingTime = false;
-
-	/**
-	 * This is used to copy the incoming element because it can be put into several window
-	 * buffers.
-	 */
-	private TypeSerializer<IN> inputSerializer;
-
-	/**
-	 * For serializing the key in checkpoints.
-	 */
-	private final TypeSerializer<K> keySerializer;
-
-	/**
-	 * For serializing the window in checkpoints.
-	 */
-	private final TypeSerializer<W> windowSerializer;
-
-	// ------------------------------------------------------------------------
-	// State that is not checkpointed
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Processing time timers that are currently in-flight.
-	 */
-	private transient Map<Long, Set<Context>> processingTimeTimers;
-
-	/**
-	 * Current waiting watermark callbacks.
-	 */
-	private transient Map<Long, Set<Context>> watermarkTimers;
-
-	/**
-	 * This is given to the {@code WindowFunction} for emitting elements with a given timestamp.
-	 */
-	protected transient TimestampedCollector<OUT> timestampedCollector;
-
-	// ------------------------------------------------------------------------
-	// State that needs to be checkpointed
-	// ------------------------------------------------------------------------
-
-	/**
-	 * The windows (panes) that are currently in-flight. Each pane has a {@code WindowBuffer}
-	 * and a {@code TriggerContext} that stores the {@code Trigger} for that pane.
-	 */
-	protected transient Map<K, Map<W, Context>> windows;
-
-	/**
-	 * Creates a new {@code WindowOperator} based on the given policies and user functions.
-	 */
-	public WindowOperator(WindowAssigner<? super IN, W> windowAssigner,
-			TypeSerializer<W> windowSerializer,
-			KeySelector<IN, K> keySelector,
-			TypeSerializer<K> keySerializer,
-			WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> windowBufferFactory,
-			WindowFunction<IN, OUT, K, W> windowFunction,
-			Trigger<? super IN, ? super W> trigger) {
-
-		super(windowFunction);
-
-		this.windowAssigner = requireNonNull(windowAssigner);
-		this.windowSerializer = windowSerializer;
-		this.keySelector = requireNonNull(keySelector);
-		this.keySerializer = requireNonNull(keySerializer);
-
-		this.windowBufferFactory = requireNonNull(windowBufferFactory);
-		this.trigger = requireNonNull(trigger);
-
-		setChainingStrategy(ChainingStrategy.ALWAYS);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public final void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
-		inputSerializer = (TypeSerializer<IN>) type.createSerializer(executionConfig);
-	}
-
-	@Override
-	public final void open() throws Exception {
-		super.open();
-
-		timestampedCollector = new TimestampedCollector<>(output);
-
-		if (inputSerializer == null) {
-			throw new IllegalStateException("Input serializer was not set.");
-		}
-
-		windowBufferFactory.setRuntimeContext(getRuntimeContext());
-		windowBufferFactory.open(getUserFunctionParameters());
-
-
-		// these could already be initialized from restoreState()
-		if (watermarkTimers == null) {
-			watermarkTimers = new HashMap<>();
-		}
-		if (processingTimeTimers == null) {
-			processingTimeTimers = new HashMap<>();
-		}
-		if (windows == null) {
-			windows = new HashMap<>();
-		}
-
-		// re-register timers that this window context had set
-		for (Map.Entry<K, Map<W, Context>> entry: windows.entrySet()) {
-			Map<W, Context> keyWindows = entry.getValue();
-			for (Context context: keyWindows.values()) {
-				if (context.processingTimeTimer > 0) {
-					Set<Context> triggers = processingTimeTimers.get(context.processingTimeTimer);
-					if (triggers == null) {
-						getRuntimeContext().registerTimer(context.processingTimeTimer, WindowOperator.this);
-						triggers = new HashSet<>();
-						processingTimeTimers.put(context.processingTimeTimer, triggers);
-					}
-					triggers.add(context);
-				}
-				if (context.watermarkTimer > 0) {
-					Set<Context> triggers = watermarkTimers.get(context.watermarkTimer);
-					if (triggers == null) {
-						triggers = new HashSet<>();
-						watermarkTimers.put(context.watermarkTimer, triggers);
-					}
-					triggers.add(context);
-				}
-
-			}
-		}
-	}
-
-	@Override
-	public final void close() throws Exception {
-		super.close();
-		// emit the elements that we still keep
-		for (Map.Entry<K, Map<W, Context>> entry: windows.entrySet()) {
-			Map<W, Context> keyWindows = entry.getValue();
-			for (Context window: keyWindows.values()) {
-				emitWindow(window);
-			}
-		}
-		windows.clear();
-		windowBufferFactory.close();
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public final void processElement(StreamRecord<IN> element) throws Exception {
-		if (setProcessingTime) {
-			element.replace(element.getValue(), System.currentTimeMillis());
-		}
-
-		Collection<W> elementWindows = windowAssigner.assignWindows(element.getValue(), element.getTimestamp());
-
-		K key = keySelector.getKey(element.getValue());
-
-		Map<W, Context> keyWindows = windows.get(key);
-		if (keyWindows == null) {
-			keyWindows = new HashMap<>();
-			windows.put(key, keyWindows);
-		}
-
-		for (W window: elementWindows) {
-			Context context = keyWindows.get(window);
-			if (context == null) {
-				WindowBuffer<IN> windowBuffer = windowBufferFactory.create();
-				context = new Context(key, window, windowBuffer);
-				keyWindows.put(window, context);
-			}
-			StreamRecord<IN> elementCopy = new StreamRecord<>(inputSerializer.copy(element.getValue()), element.getTimestamp());
-			context.windowBuffer.storeElement(elementCopy);
-			Trigger.TriggerResult triggerResult = trigger.onElement(elementCopy.getValue(), elementCopy.getTimestamp(), window, context);
-			processTriggerResult(triggerResult, key, window);
-		}
-	}
-
-	protected void emitWindow(Context context) throws Exception {
-		timestampedCollector.setTimestamp(context.window.maxTimestamp());
-
-		userFunction.apply(context.key,
-				context.window,
-				context.windowBuffer.getUnpackedElements(),
-				timestampedCollector);
-	}
-
-	private void processTriggerResult(Trigger.TriggerResult triggerResult, K key, W window) throws Exception {
-		switch (triggerResult) {
-			case FIRE: {
-				Map<W, Context> keyWindows = windows.get(key);
-				if (keyWindows == null) {
-					LOG.debug("Window {} for key {} already gone.", window, key);
-					return;
-				}
-				Context context = keyWindows.get(window);
-				if (context == null) {
-					LOG.debug("Window {} for key {} already gone.", window, key);
-					return;
-				}
-
-
-				emitWindow(context);
-				break;
-			}
-
-			case FIRE_AND_PURGE: {
-				Map<W, Context> keyWindows = windows.get(key);
-				if (keyWindows == null) {
-					LOG.debug("Window {} for key {} already gone.", window, key);
-					return;
-				}
-				Context context = keyWindows.remove(window);
-				if (context == null) {
-					LOG.debug("Window {} for key {} already gone.", window, key);
-					return;
-				}
-				if (keyWindows.isEmpty()) {
-					windows.remove(key);
-				}
-
-				emitWindow(context);
-				break;
-			}
-
-			case CONTINUE:
-				// ingore
-		}
-	}
-
-	@Override
-	public final void processWatermark(Watermark mark) throws Exception {
-		Set<Long> toRemove = new HashSet<>();
-
-		for (Map.Entry<Long, Set<Context>> triggers: watermarkTimers.entrySet()) {
-			if (triggers.getKey() <= mark.getTimestamp()) {
-				for (Context context: triggers.getValue()) {
-					Trigger.TriggerResult triggerResult = context.onEventTime(triggers.getKey());
-					processTriggerResult(triggerResult, context.key, context.window);
-				}
-				toRemove.add(triggers.getKey());
-			}
-		}
-
-		for (Long l: toRemove) {
-			watermarkTimers.remove(l);
-		}
-		output.emitWatermark(mark);
-	}
-
-	@Override
-	public final void trigger(long time) throws Exception {
-		Set<Long> toRemove = new HashSet<>();
-
-		for (Map.Entry<Long, Set<Context>> triggers: processingTimeTimers.entrySet()) {
-			if (triggers.getKey() < time) {
-				for (Context context: triggers.getValue()) {
-					Trigger.TriggerResult triggerResult = context.onProcessingTime(time);
-					processTriggerResult(triggerResult, context.key, context.window);
-				}
-				toRemove.add(triggers.getKey());
-			}
-		}
-
-		for (Long l: toRemove) {
-			processingTimeTimers.remove(l);
-		}
-	}
-
-	/**
-	 * A context object that is given to {@code Trigger} functions to allow them to register
-	 * timer/watermark callbacks.
-	 */
-	protected class Context implements Trigger.TriggerContext {
-		protected K key;
-		protected W window;
-
-		protected WindowBuffer<IN> windowBuffer;
-
-		protected HashMap<String, Serializable> state;
-
-		// use these to only allow one timer in flight at a time of each type
-		// if the trigger registers another timer this value here will be overwritten,
-		// the timer is not removed from the set of in-flight timers to improve performance.
-		// When a trigger fires it is just checked against the last timer that was set.
-		protected long watermarkTimer;
-		protected long processingTimeTimer;
-
-		public Context(K key,
-				W window,
-				WindowBuffer<IN> windowBuffer) {
-			this.key = key;
-			this.window = window;
-			this.windowBuffer = windowBuffer;
-			state = new HashMap<>();
-
-			this.watermarkTimer = -1;
-			this.processingTimeTimer = -1;
-		}
-
-		/**
-		 * Constructs a new {@code Context} by reading from a {@link DataInputView} that
-		 * contains a serialized context that we wrote in
-		 * {@link #writeToState(StateBackend.CheckpointStateOutputView)}
-		 */
-		@SuppressWarnings("unchecked")
-		protected Context(DataInputView in) throws Exception {
-			this.key = keySerializer.deserialize(in);
-			this.window = windowSerializer.deserialize(in);
-			this.watermarkTimer = in.readLong();
-			this.processingTimeTimer = in.readLong();
-
-			int stateSize = in.readInt();
-			byte[] stateData = new byte[stateSize];
-			in.read(stateData);
-			ByteArrayInputStream bais = new ByteArrayInputStream(stateData);
-			state = (HashMap<String, Serializable>) SerializationUtils.deserialize(bais);
-
-			this.windowBuffer = windowBufferFactory.create();
-			int numElements = in.readInt();
-			MultiplexingStreamRecordSerializer<IN> recordSerializer = new MultiplexingStreamRecordSerializer<>(inputSerializer);
-			for (int i = 0; i < numElements; i++) {
-				windowBuffer.storeElement(recordSerializer.deserialize(in).<IN>asRecord());
-			}
-		}
-
-		/**
-		 * Writes the {@code Context} to the given state checkpoint output.
-		 */
-		protected void writeToState(StateBackend.CheckpointStateOutputView out) throws IOException {
-			keySerializer.serialize(key, out);
-			windowSerializer.serialize(window, out);
-			out.writeLong(watermarkTimer);
-			out.writeLong(processingTimeTimer);
-
-			ByteArrayOutputStream baos = new ByteArrayOutputStream();
-			SerializationUtils.serialize(state, baos);
-			out.writeInt(baos.size());
-			out.write(baos.toByteArray(), 0, baos.size());
-
-			MultiplexingStreamRecordSerializer<IN> recordSerializer = new MultiplexingStreamRecordSerializer<>(inputSerializer);
-			out.writeInt(windowBuffer.size());
-			for (StreamRecord<IN> element: windowBuffer.getElements()) {
-				recordSerializer.serialize(element, out);
-			}
-		}
-
-		@SuppressWarnings("unchecked")
-		public <S extends Serializable> OperatorState<S> getKeyValueState(final String name, final S defaultState) {
-			return new OperatorState<S>() {
-				@Override
-				public S value() throws IOException {
-					Serializable value = state.get(name);
-					if (value == null) {
-						state.put(name, defaultState);
-						value = defaultState;
-					}
-					return (S) value;
-				}
-
-				@Override
-				public void update(S value) throws IOException {
-					state.put(name, value);
-				}
-			};
-		}
-
-		@Override
-		public void registerProcessingTimeTimer(long time) {
-			if (this.processingTimeTimer == time) {
-				// we already have set a trigger for that time
-				return;
-			}
-			Set<Context> triggers = processingTimeTimers.get(time);
-			if (triggers == null) {
-				getRuntimeContext().registerTimer(time, WindowOperator.this);
-				triggers = new HashSet<>();
-				processingTimeTimers.put(time, triggers);
-			}
-			this.processingTimeTimer = time;
-			triggers.add(this);
-		}
-
-		@Override
-		public void registerEventTimeTimer(long time) {
-			if (watermarkTimer == time) {
-				// we already have set a trigger for that time
-				return;
-			}
-			Set<Context> triggers = watermarkTimers.get(time);
-			if (triggers == null) {
-				triggers = new HashSet<>();
-				watermarkTimers.put(time, triggers);
-			}
-			this.watermarkTimer = time;
-			triggers.add(this);
-		}
-
-		public Trigger.TriggerResult onProcessingTime(long time) throws Exception {
-			if (time == processingTimeTimer) {
-				return trigger.onProcessingTime(time, this);
-			} else {
-				return Trigger.TriggerResult.CONTINUE;
-			}
-		}
-
-		public Trigger.TriggerResult onEventTime(long time) throws Exception {
-			if (time == watermarkTimer) {
-				return trigger.onEventTime(time, this);
-			} else {
-				return Trigger.TriggerResult.CONTINUE;
-			}
-		}
-	}
-
-	/**
-	 * 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 WindowOperator<K, IN, OUT, W> enableSetProcessingTime(boolean setProcessingTime) {
-		this.setProcessingTime = setProcessingTime;
-		return this;
-	}
-
-	@Override
-	public final void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) {
-		if (userFunction instanceof OutputTypeConfigurable) {
-			@SuppressWarnings("unchecked")
-			OutputTypeConfigurable<OUT> typeConfigurable = (OutputTypeConfigurable<OUT>) userFunction;
-			typeConfigurable.setOutputType(outTypeInfo, executionConfig);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpointing
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
-		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
-
-		// we write the panes with the key/value maps into the stream
-		StateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
-
-		int numKeys = windows.size();
-		out.writeInt(numKeys);
-
-		for (Map.Entry<K, Map<W, Context>> keyWindows: windows.entrySet()) {
-			int numWindows = keyWindows.getValue().size();
-			out.writeInt(numWindows);
-			for (Context context: keyWindows.getValue().values()) {
-				context.writeToState(out);
-			}
-		}
-
-		taskState.setOperatorState(out.closeAndGetHandle());
-		return taskState;
-	}
-
-	@Override
-	public void restoreState(StreamTaskState taskState) throws Exception {
-		super.restoreState(taskState);
-
-
-		@SuppressWarnings("unchecked")
-		StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
-		DataInputView in = inputState.getState(getUserCodeClassloader());
-
-		int numKeys = in.readInt();
-		this.windows = new HashMap<>(numKeys);
-		this.processingTimeTimers = new HashMap<>();
-		this.watermarkTimers = new HashMap<>();
-
-		for (int i = 0; i < numKeys; i++) {
-			int numWindows = in.readInt();
-			for (int j = 0; j < numWindows; j++) {
-				Context context = new Context(in);
-				Map<W, Context> keyWindows = windows.get(context.key);
-				if (keyWindows == null) {
-					keyWindows = new HashMap<>(numWindows);
-					windows.put(context.key, keyWindows);
-				}
-				keyWindows.put(context.window, context);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	// Getters for testing
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	public boolean isSetProcessingTime() {
-		return setProcessingTime;
-	}
-
-	@VisibleForTesting
-	public Trigger<? super IN, ? super W> getTrigger() {
-		return trigger;
-	}
-
-	@VisibleForTesting
-	public KeySelector<IN, K> getKeySelector() {
-		return keySelector;
-	}
-
-	@VisibleForTesting
-	public WindowAssigner<? super IN, W> getWindowAssigner() {
-		return windowAssigner;
-	}
-
-	@VisibleForTesting
-	public WindowBufferFactory<? super IN, ? extends WindowBuffer<IN>> getWindowBufferFactory() {
-		return windowBufferFactory;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java
deleted file mode 100644
index 28365e1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/EvictingWindowBuffer.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.operators.windowing.buffers;
-
-/**
- * A {@code WindowBuffer} that can also evict elements from the buffer. The order in which
- * the elements are added is preserved. Elements can only be evicted started from the beginning of
- * the buffer.
- *
- * @param <T> The type of elements that this {@code WindowBuffer} can store.
- */
-
-public interface EvictingWindowBuffer<T> extends WindowBuffer<T> {
-
-	/**
-	 * Removes the given number of elements, starting from the beginning.
-	 * @param count The number of elements to remove.
-	 */
-	void removeElements(int count);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java
deleted file mode 100644
index f9f8b26..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/HeapWindowBuffer.java
+++ /dev/null
@@ -1,92 +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.buffers;
-
-import com.google.common.base.Function;
-import com.google.common.collect.FluentIterable;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import java.util.ArrayDeque;
-
-/**
- * An {@link EvictingWindowBuffer} that stores elements on the Java Heap.
- *
- * @param <T> The type of elements that this {@code WindowBuffer} can store.
- */
-public class HeapWindowBuffer<T> implements EvictingWindowBuffer<T> {
-	private static final long serialVersionUID = 1L;
-
-	private ArrayDeque<StreamRecord<T>> elements;
-
-	protected HeapWindowBuffer() {
-		this.elements = new ArrayDeque<>();
-	}
-
-	@Override
-	public void storeElement(StreamRecord<T> element) {
-		elements.add(element);
-	}
-
-	@Override
-	public void removeElements(int count) {
-		// TODO determine if this can be done in a better way
-		for (int i = 0; i < count; i++) {
-			elements.removeFirst();
-		}
-	}
-
-	@Override
-	public Iterable<StreamRecord<T>> getElements() {
-		return elements;
-	}
-
-	@Override
-	public Iterable<T> getUnpackedElements() {
-		return FluentIterable.from(elements).transform(new Function<StreamRecord<T>, T>() {
-			@Override
-			public T apply(StreamRecord<T> record) {
-				return record.getValue();
-			}
-		});
-	}
-
-	@Override
-	public int size() {
-		return elements.size();
-	}
-
-	public static class Factory<T> implements WindowBufferFactory<T, HeapWindowBuffer<T>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void setRuntimeContext(RuntimeContext ctx) {}
-
-		@Override
-		public void open(Configuration config) {}
-
-		@Override
-		public void close() {}
-
-		@Override
-		public HeapWindowBuffer<T> create() {
-			return new HeapWindowBuffer<>();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java
deleted file mode 100644
index 37be8f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/PreAggregatingHeapWindowBuffer.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.operators.windowing.buffers;
-
-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.runtime.streamrecord.StreamRecord;
-
-import java.util.Collections;
-
-/**
- * An {@link WindowBuffer} that stores elements on the Java Heap. This buffer uses a
- * {@link ReduceFunction} to pre-aggregate elements that are added to the buffer.
- *
- * @param <T> The type of elements that this {@code WindowBuffer} can store.
- */
-
-public class PreAggregatingHeapWindowBuffer<T> implements WindowBuffer<T> {
-	private static final long serialVersionUID = 1L;
-
-	private final ReduceFunction<T> reduceFunction;
-	private transient StreamRecord<T> data;
-
-	protected PreAggregatingHeapWindowBuffer(ReduceFunction<T> reduceFunction) {
-		this.reduceFunction = reduceFunction;
-	}
-
-	@Override
-	public void storeElement(StreamRecord<T> element) throws Exception {
-		if (data == null) {
-			data = new StreamRecord<>(element.getValue(), element.getTimestamp());
-		} else {
-			data.replace(reduceFunction.reduce(data.getValue(), element.getValue()));
-		}
-	}
-
-	@Override
-	public Iterable<StreamRecord<T>> getElements() {
-		return Collections.singleton(data);
-	}
-
-	@Override
-	public Iterable<T> getUnpackedElements() {
-		return Collections.singleton(data.getValue());
-	}
-
-	@Override
-	public int size() {
-		return 1;
-	}
-
-	public static class Factory<T> implements WindowBufferFactory<T, PreAggregatingHeapWindowBuffer<T>> {
-		private static final long serialVersionUID = 1L;
-
-		private final ReduceFunction<T> reduceFunction;
-
-		public Factory(ReduceFunction<T> reduceFunction) {
-			this.reduceFunction = reduceFunction;
-		}
-
-		@Override
-		public void setRuntimeContext(RuntimeContext ctx) {
-			FunctionUtils.setFunctionRuntimeContext(reduceFunction, ctx);
-		}
-
-		@Override
-		public void open(Configuration config) throws Exception {
-			FunctionUtils.openFunction(reduceFunction, config);
-		}
-
-		@Override
-		public void close() throws Exception {
-			FunctionUtils.closeFunction(reduceFunction);
-		}
-
-		@Override
-		public PreAggregatingHeapWindowBuffer<T> create() {
-			return new PreAggregatingHeapWindowBuffer<>(reduceFunction);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.java
deleted file mode 100644
index b111667..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBuffer.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.runtime.operators.windowing.buffers;
-
-
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import java.io.Serializable;
-
-/**
- * A {@code WindowBuffer} is used by
- * {@link org.apache.flink.streaming.runtime.operators.windowing.WindowOperator} to store
- * the elements of one pane.
- *
- * <p>
- * A pane is the bucket of elements that have the same key (assigned by the
- * {@link org.apache.flink.api.java.functions.KeySelector}) and same {@link Window}. An element can
- * be in multiple panes of it was assigned to multiple windows by the
- * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. These panes all
- * have their own instance of the {@code Evictor}.
- *
- * @param <T> The type of elements that this {@code WindowBuffer} can store.
- */
-public interface WindowBuffer<T> extends Serializable {
-
-	/**
-	 * Adds the element to the buffer.
-	 *
-	 * @param element The element to add.
-	 */
-	void storeElement(StreamRecord<T> element) throws Exception;
-
-	/**
-	 * Returns all elements that are currently in the buffer.
-	 */
-	Iterable<StreamRecord<T>> getElements();
-
-	/**
-	 * Returns all elements that are currently in the buffer. This will unwrap the contained
-	 * elements from their {@link StreamRecord}.
-	 */
-	Iterable<T> getUnpackedElements();
-
-	/**
-	 * Returns the number of elements that are currently in the buffer.
-	 */
-	int size();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java
deleted file mode 100644
index 4bcdf09..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/buffers/WindowBufferFactory.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.operators.windowing.buffers;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.Configuration;
-
-import java.io.Serializable;
-
-/**
- * A factory for {@link WindowBuffer WindowBuffers}.
- *
- * @param <T> The type of elements that the created {@code WindowBuffer} can store.
- * @param <B> The type of the created {@code WindowBuffer}
- */
-public interface WindowBufferFactory<T, B extends WindowBuffer<T>> extends Serializable {
-
-	/**
-	 * Sets the {@link RuntimeContext} that is used to initialize eventual user functions
-	 * inside the created buffers.
-	 */
-	void setRuntimeContext(RuntimeContext ctx);
-
-	/**
-	 * Calls {@code open()} on eventual user functions inside the buffer.
-	 */
-	void open(Configuration config) throws Exception;
-
-	/**
-	 * Calls {@code close()} on eventual user functions inside the buffer.
-	 */
-
-	void close() throws Exception;
-
-	/**
-	 * Creates a new {@code WindowBuffer}.
-	 */
-	B create();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/package-info.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/package-info.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/package-info.java
deleted file mode 100644
index 55749a1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * This package contains the operators that implement the various window operations
- * on data streams. 
- */
-package org.apache.flink.streaming.runtime.operators.windowing;

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java
deleted file mode 100644
index f3d851c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/BroadcastPartitioner.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner that selects all the output channels.
- *
- * @param <T> Type of the elements in the Stream being broadcast
- */
-public class BroadcastPartitioner<T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	int[] returnArray;
-	boolean set;
-	int setNumber;
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
-			int numberOfOutputChannels) {
-		if (set && setNumber == numberOfOutputChannels) {
-			return returnArray;
-		} else {
-			this.returnArray = new int[numberOfOutputChannels];
-			for (int i = 0; i < numberOfOutputChannels; i++) {
-				returnArray[i] = i;
-			}
-			set = true;
-			setNumber = numberOfOutputChannels;
-			return returnArray;
-		}
-	}
-
-	@Override
-	public StreamPartitioner<T> copy() {
-		return this;
-	}
-
-	@Override
-	public String toString() {
-		return "BROADCAST";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
deleted file mode 100644
index 7bb9480..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/CustomPartitionerWrapper.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner that selects the channel with a user defined partitioner function on a key.
- *
- * @param <K>
- *            Type of the key
- * @param <T>
- *            Type of the data
- */
-public class CustomPartitionerWrapper<K, T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	private int[] returnArray = new int[1];
-	Partitioner<K> partitioner;
-	KeySelector<T, K> keySelector;
-
-	public CustomPartitionerWrapper(Partitioner<K> partitioner, KeySelector<T, K> keySelector) {
-		this.partitioner = partitioner;
-		this.keySelector = keySelector;
-	}
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record, int numberOfOutputChannels) {
-
-		K key = null;
-		try {
-			key = keySelector.getKey(record.getInstance().getValue());
-		} catch (Exception e) {
-			throw new RuntimeException("Could not extract key from " + record.getInstance(), e);
-		}
-
-		returnArray[0] = partitioner.partition(key,
-				numberOfOutputChannels);
-
-		return returnArray;
-	}
-
-	@Override
-	public StreamPartitioner<T> copy() {
-		return this;
-	}
-
-	@Override
-	public String toString() {
-		return "CUSTOM";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java
deleted file mode 100644
index 4fb460c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ForwardPartitioner.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner that forwards elements only to the locally running downstream operation.
- * 
- * @param <T> Type of the elements in the Stream
- */
-public class ForwardPartitioner<T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	private int[] returnArray = new int[] {0};
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record, int numberOfOutputChannels) {
-		return returnArray;
-	}
-	
-	public StreamPartitioner<T> copy() {
-		return this;
-	}
-	
-	@Override
-	public String toString() {
-		return "FORWARD";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java
deleted file mode 100644
index b19fb41..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/GlobalPartitioner.java
+++ /dev/null
@@ -1,48 +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.partitioner;
-
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner that sends all elements to the downstream operator with subtask ID=0;
- *
- * @param <T> Type of the elements in the Stream being partitioned
- */
-public class GlobalPartitioner<T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	private int[] returnArray = new int[] { 0 };
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
-			int numberOfOutputChannels) {
-		return returnArray;
-	}
-
-	@Override
-	public StreamPartitioner<T> copy() {
-		return this;
-	}
-
-	@Override
-	public String toString() {
-		return "GLOBAL";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java
deleted file mode 100644
index a3f5158..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/HashPartitioner.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner selects the target channel based on the hash value of a key from a
- * {@link KeySelector}.
- *
- * @param <T> Type of the elements in the Stream being partitioned
- */
-public class HashPartitioner<T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	private int[] returnArray = new int[1];
-	KeySelector<T, ?> keySelector;
-
-	public HashPartitioner(KeySelector<T, ?> keySelector) {
-		this.keySelector = keySelector;
-	}
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
-			int numberOfOutputChannels) {
-		Object key;
-		try {
-			key = keySelector.getKey(record.getInstance().getValue());
-		} catch (Exception e) {
-			throw new RuntimeException("Could not extract key from " + record.getInstance().getValue(), e);
-		}
-		returnArray[0] = Math.abs(key.hashCode() % numberOfOutputChannels);
-
-		return returnArray;
-	}
-
-	@Override
-	public StreamPartitioner<T> copy() {
-		return this;
-	}
-
-	@Override
-	public String toString() {
-		return "HASH";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
deleted file mode 100644
index 2dfff0e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/RebalancePartitioner.java
+++ /dev/null
@@ -1,49 +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.partitioner;
-
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner that distributes the data equally by cycling through the output
- * channels.
- * 
- * @param <T> Type of the elements in the Stream being rebalanced
- */
-public class RebalancePartitioner<T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	private int[] returnArray = new int[] {-1};
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
-			int numberOfOutputChannels) {
-		this.returnArray[0] = (this.returnArray[0] + 1) % numberOfOutputChannels;
-		return this.returnArray;
-	}
-	
-	public StreamPartitioner<T> copy() {
-		return this;
-	}
-	
-	@Override
-	public String toString() {
-		return "REBALANCE";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java
deleted file mode 100644
index 93c6f9c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/ShufflePartitioner.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.partitioner;
-
-import java.util.Random;
-
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Partitioner that distributes the data equally by selecting one output channel
- * randomly.
- * 
- * @param <T>
- *            Type of the Tuple
- */
-public class ShufflePartitioner<T> extends StreamPartitioner<T> {
-	private static final long serialVersionUID = 1L;
-
-	private Random random = new Random();
-
-	private int[] returnArray = new int[1];
-
-	@Override
-	public int[] selectChannels(SerializationDelegate<StreamRecord<T>> record,
-			int numberOfOutputChannels) {
-		returnArray[0] = random.nextInt(numberOfOutputChannels);
-		return returnArray;
-	}
-
-	@Override
-	public StreamPartitioner<T> copy() {
-		return new ShufflePartitioner<T>();
-	}
-
-	@Override
-	public String toString() {
-		return "SHUFFLE";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java
deleted file mode 100644
index 4ef360d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/partitioner/StreamPartitioner.java
+++ /dev/null
@@ -1,30 +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.partitioner;
-
-import java.io.Serializable;
-
-import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public abstract class StreamPartitioner<T> implements
-		ChannelSelector<SerializationDelegate<StreamRecord<T>>>, Serializable {
-	private static final long serialVersionUID = 1L;
-
-	public abstract StreamPartitioner<T> copy();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
deleted file mode 100644
index d4363cd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUStreamRecord<?>WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.streamrecord;
-
-import com.google.common.base.Preconditions;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-import java.io.IOException;
-
-/**
- * Serializer for {@link StreamRecord} and {@link org.apache.flink.streaming.api.watermark.Watermark}. This does not behave like a normal
- * {@link TypeSerializer}, instead, this is only used at the
- * {@link org.apache.flink.streaming.runtime.tasks.StreamTask} level for transmitting
- * {@link StreamRecord StreamRecords} and {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks}. This serializer
- * can handle both of them, therefore it returns {@link Object} the result has
- * to be cast to the correct type.
- *
- * @param <T> The type of value in the {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}
- */
-public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<StreamElement> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final long IS_WATERMARK = Long.MIN_VALUE;
-	
-	private final TypeSerializer<T> typeSerializer;
-
-	
-	public MultiplexingStreamRecordSerializer(TypeSerializer<T> serializer) {
-		if (serializer instanceof MultiplexingStreamRecordSerializer || serializer instanceof StreamRecordSerializer) {
-			throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer);
-		}
-		this.typeSerializer = Preconditions.checkNotNull(serializer);
-	}
-	
-	
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public TypeSerializer<StreamElement> duplicate() {
-		TypeSerializer<T> copy = typeSerializer.duplicate();
-		return (copy == typeSerializer) ? this : new MultiplexingStreamRecordSerializer<T>(copy);
-	}
-
-	@Override
-	public StreamRecord<T> createInstance() {
-		return new StreamRecord<T>(typeSerializer.createInstance(), 0L);
-	}
-
-	@Override
-	public StreamElement copy(StreamElement from) {
-		// we can reuse the timestamp since Instant is immutable
-		if (from.isRecord()) {
-			StreamRecord<T> fromRecord = from.asRecord();
-			return new StreamRecord<T>(typeSerializer.copy(fromRecord.getValue()), fromRecord.getTimestamp());
-		}
-		else if (from.isWatermark()) {
-			// is immutable
-			return from;
-		}
-		else {
-			throw new RuntimeException("Cannot copy " + from);
-		}
-	}
-
-	@Override
-	public StreamElement copy(StreamElement from, StreamElement reuse) {
-		if (from.isRecord() && reuse.isRecord()) {
-			StreamRecord<T> fromRecord = from.asRecord();
-			StreamRecord<T> reuseRecord = reuse.asRecord();
-
-			T valueCopy = typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue());
-			reuseRecord.replace(valueCopy, fromRecord.getTimestamp());
-			return reuse;
-		}
-		else if (from.isWatermark()) {
-			// is immutable
-			return from;
-		}
-		else {
-			throw new RuntimeException("Cannot copy " + from + " -> " + reuse);
-		}
-	}
-
-	@Override
-	public int getLength() {
-		return -1;
-	}
-
-	@Override
-	public void serialize(StreamElement value, DataOutputView target) throws IOException {
-		if (value.isRecord()) {
-			StreamRecord<T> record = value.asRecord();
-			target.writeLong(record.getTimestamp());
-			typeSerializer.serialize(record.getValue(), target);
-		}
-		else if (value.isWatermark()) {
-			target.writeLong(IS_WATERMARK);
-			target.writeLong(value.asWatermark().getTimestamp());
-		}
-		else {
-			throw new RuntimeException();
-		}
-	}
-	
-	@Override
-	public StreamElement deserialize(DataInputView source) throws IOException {
-		long millis = source.readLong();
-
-		if (millis == IS_WATERMARK) {
-			return new Watermark(source.readLong());
-		}
-		else {
-			T element = typeSerializer.deserialize(source);
-			return new StreamRecord<T>(element, millis);
-		}
-	}
-
-	@Override
-	public StreamElement deserialize(StreamElement reuse, DataInputView source) throws IOException {
-		long millis = source.readLong();
-
-		if (millis == IS_WATERMARK) {
-			return new Watermark(source.readLong());
-		}
-		else {
-			StreamRecord<T> reuseRecord = reuse.asRecord();
-			T element = typeSerializer.deserialize(reuseRecord.getValue(), source);
-			reuseRecord.replace(element, millis);
-			return reuse;
-		}
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		long millis = source.readLong();
-		target.writeLong(millis);
-
-		if (millis == IS_WATERMARK) {
-			target.writeLong(source.readLong());
-		} else {
-			typeSerializer.copy(source, target);
-		}
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof MultiplexingStreamRecordSerializer) {
-			MultiplexingStreamRecordSerializer<?> other = (MultiplexingStreamRecordSerializer<?>) obj;
-
-			return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof MultiplexingStreamRecordSerializer;
-	}
-
-	@Override
-	public int hashCode() {
-		return typeSerializer.hashCode();
-	}
-}


[41/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
deleted file mode 100644
index 033e84f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ /dev/null
@@ -1,648 +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.FoldFunction;
-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.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
-import org.apache.flink.streaming.api.functions.windowing.FoldWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-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.SlidingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.api.windowing.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.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-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 {@code WindowedStream} represents a data stream where elements are grouped by
- * key, and for each key, the stream of elements is split into windows based on a
- * {@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 WindowedStream} is purely and API construct, during runtime
- * the {@code WindowedStream} will be collapsed together with the
- * {@code KeyedStream} 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 WindowedStream<T, K, W extends Window> {
-
-	/** The keyed data stream that is windowed by this stream */
-	private final KeyedStream<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 WindowedStream(KeyedStream<T, K> input,
-			WindowAssigner<? super T, W> windowAssigner) {
-		this.input = input;
-		this.windowAssigner = windowAssigner;
-		this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
-	}
-
-	/**
-	 * Sets the {@code Trigger} that should be used to trigger window emission.
-	 */
-	public WindowedStream<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 WindowedStream<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 SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> function) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "Reduce at " + callLocation;
-
-		SingleOutputStreamOperator<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;
-
-		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
-
-		if (evictor != null) {
-			operator = new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					keySel,
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					new ReduceWindowFunction<K, W, T>(function),
-					trigger,
-					evictor).enableSetProcessingTime(setProcessingTime);
-
-		} 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,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					keySel,
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
-					new ReduceWindowFunction<K, W, T>(function),
-					trigger).enableSetProcessingTime(setProcessingTime);
-		}
-
-		return input.transform(opName, input.getType(), operator);
-	}
-
-	/**
-	 * Applies the given fold function to each 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.
-	 *
-	 * @param function The fold function.
-	 * @return The data stream that is the result of applying the fold function to the window.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-
-		TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
-				Utils.getCallLocationName(), true);
-
-		return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
-	}
-
-	/**
-	 * Applies the given fold function to each 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.
-	 *
-	 * @param function The fold function.
-	 * @return The data stream that is the result of applying the fold function to the window.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-		return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <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> SingleOutputStreamOperator<R, ?> apply(WindowFunction<T, R, K, W> function) {
-		TypeInformation<T> inType = input.getType();
-		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
-				function, WindowFunction.class, true, true, inType, null, false);
-
-		return apply(function, resultType);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <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.
-	 * @param resultType Type information for the result type of the window function
-	 * @return The data stream that is the result of applying the window function to the window.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> apply(WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "WindowApply at " + callLocation;
-
-		SingleOutputStreamOperator<R, ?> result = createFastTimeOperatorIfValid(function, resultType, udfName);
-		if (result != null) {
-			return result;
-		}
-
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-		KeySelector<T, K> keySel = input.getKeySelector();
-
-		WindowOperator<K, T, R, W> operator;
-
-		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
-
-		if (evictor != null) {
-			operator = new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					keySel,
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger,
-					evictor).enableSetProcessingTime(setProcessingTime);
-
-		} else {
-			operator = new WindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					keySel,
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger).enableSetProcessingTime(setProcessingTime);
-		}
-
-		return input.transform(opName, resultType, operator);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <p>
-	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-	 *
-	 * @param preAggregator The reduce function that is used for 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> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, WindowFunction<T, R, K, W> function) {
-		TypeInformation<T> inType = input.getType();
-		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
-				function, WindowFunction.class, true, true, inType, null, false);
-
-		return apply(preAggregator, function, resultType);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <p>
-	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-	 *
-	 * @param preAggregator The reduce function that is used for pre-aggregation
-	 * @param function The window function.
-	 * @param resultType Type information for the result type of the window function
-	 * @return The data stream that is the result of applying the window function to the window.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) {
-		//clean the closures
-		function = input.getExecutionEnvironment().clean(function);
-		preAggregator = input.getExecutionEnvironment().clean(preAggregator);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "WindowApply at " + callLocation;
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-		KeySelector<T, K> keySel = input.getKeySelector();
-
-		OneInputStreamOperator<T, R> operator;
-
-		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
-
-		if (evictor != null) {
-			operator = new EvictingWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					keySel,
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger,
-					evictor).enableSetProcessingTime(setProcessingTime);
-
-		} else {
-			operator = new WindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					keySel,
-					input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-					new PreAggregatingHeapWindowBuffer.Factory<>(preAggregator),
-					function,
-					trigger).enableSetProcessingTime(setProcessingTime);
-		}
-
-		return input.transform(opName, resultType, operator);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Aggregations on the keyed windows
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Applies an aggregation that sums every window of the data stream at the
-	 * given position.
-	 *
-	 * @param positionToSum The position in the tuple/array to sum
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
-		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that sums every window of the pojo data stream at
-	 * the given field for every window.
-	 *
-	 * <p>
-	 * A field expression is either
-	 * the name of a public field or a getter method with parentheses of the
-	 * stream's underlying type. A dot can be used to drill down into objects,
-	 * as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field to sum
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(String field) {
-		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum value of every window
-	 * of the data stream at the given position.
-	 *
-	 * @param positionToMin The position to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
-		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum value of the pojo data
-	 * stream at the given field expression for every window.
-	 *
-	 * <p>
-	 * A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(String field) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns the first element by default.
-	 *
-	 * @param positionToMinBy
-	 *            The position to minimize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns the first element by default.
-	 *
-	 * @param positionToMinBy The position to minimize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns either the first or last one depending
-	 * on the parameter setting.
-	 *
-	 * @param positionToMinBy The position to minimize
-	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
-		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum element of the pojo
-	 * data stream by the given field expression for every window. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream DataStreams} underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @param first If True then in case of field equality the first object will be returned
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum value of every window of
-	 * the data stream at the given position.
-	 *
-	 * @param positionToMax The position to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
-		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the maximum value of the pojo data
-	 * stream at the given field expression for every window. A field expression
-	 * is either the name of a public field or a getter method with parentheses
-	 * of the {@link DataStream DataStreams} underlying type. A dot can be used to drill
-	 * down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(String field) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns the first by default.
-	 *
-	 * @param positionToMaxBy
-	 *            The position to maximize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns the first by default.
-	 *
-	 * @param positionToMaxBy
-	 *            The position to maximize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns either the first or last one depending
-	 * on the parameter setting.
-	 *
-	 * @param positionToMaxBy The position to maximize by
-	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
-		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the maximum element of the pojo
-	 * data stream by the given field expression for every window. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @param first If True then in case of field equality the first object will be returned
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
-	}
-
-	private SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregator) {
-		return reduce(aggregator);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private <R> SingleOutputStreamOperator<R, ?> createFastTimeOperatorIfValid(
-			Function function,
-			TypeInformation<R> resultType,
-			String functionName) {
-
-		if (windowAssigner instanceof SlidingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			SlidingTimeWindows timeWindows = (SlidingTimeWindows) 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(), 
-								input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-								input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-								windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-			else if (function instanceof WindowFunction) {
-				@SuppressWarnings("unchecked")
-				WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
-
-				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
-						wf, input.getKeySelector(),
-						input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-						input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-						windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-		} else if (windowAssigner instanceof TumblingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
-			TumblingTimeWindows timeWindows = (TumblingTimeWindows) 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(),
-								input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-								input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-								windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-			else if (function instanceof WindowFunction) {
-				@SuppressWarnings("unchecked")
-				WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
-
-				OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
-						wf, input.getKeySelector(),
-						input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
-						input.getType().createSerializer(getExecutionEnvironment().getConfig()),
-						windowLength, windowSlide);
-				return input.transform(opName, resultType, op);
-			}
-		}
-
-		return null;
-	}
-
-	public StreamExecutionEnvironment getExecutionEnvironment() {
-		return input.getExecutionEnvironment();
-	}
-
-	public TypeInformation<T> getInputType() {
-		return input.getType();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index f0bd174..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.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.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.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() {
-		this(null);
-	}
-
-	/**
-	 * Creates a new local stream environment that configures its local executor with the given configuration.
-	 *
-	 * @param config The configuration used to configure the local executor.
-	 */
-	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;
-	}
-
-	/**
-	 * Executes the JobGraph of the on a mini cluster of CLusterUtil with a user
-	 * specified name.
-	 * 
-	 * @param jobName
-	 *            name of the job
-	 * @return The result of the job execution, containing elapsed time and accumulators.
-	 */
-	@Override
-	public JobExecutionResult execute(String jobName) throws Exception {
-		// 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/06f6ac5d/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
deleted file mode 100644
index 02c938e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java
+++ /dev/null
@@ -1,253 +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.environment;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-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;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class RemoteStreamEnvironment extends StreamExecutionEnvironment {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(RemoteStreamEnvironment.class);
-
-	/** The hostname of the JobManager */
-	private final String host;
-
-	/** The port of the JobManager main actor system */
-	private final int port;
-
-	/** The configuration used to parametrize the client that connects to the remote cluster */
-	private final Configuration config;
-
-	/** The jar files that need to be attached to each job */
-	private final List<URL> jarFiles;
-	
-	/** The classpaths that need to be attached to each job */
-	private final List<URL> globalClasspaths;
-
-	/**
-	 * 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 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, 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) {
-		this(host, port, config, jarFiles, null);
-	}
-
-	/**
-	 * 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.
-	 * @param globalClasspaths 
-	 *            The paths of directories and JAR files that are added to each user code 
-	 *            classloader on all nodes in the cluster. Note that the paths must specify a 
-	 *            protocol (e.g. file://) and be accessible on all nodes (e.g. by means of a NFS share).
-	 *            The protocol must be supported by the {@link java.net.URLClassLoader}.
-	 */
-	public RemoteStreamEnvironment(String host, int port, Configuration config, String[] jarFiles, URL[] globalClasspaths) {
-		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.config = config == null ? new Configuration() : config;
-		this.jarFiles = new ArrayList<>(jarFiles.length);
-		for (String jarFile : jarFiles) {
-			try {
-				URL jarFileUrl = new File(jarFile).getAbsoluteFile().toURI().toURL();
-				this.jarFiles.add(jarFileUrl);
-				JobWithJars.checkJarFile(jarFileUrl);
-			} catch (MalformedURLException e) {
-				throw new IllegalArgumentException("JAR file path is invalid '" + jarFile + "'", e);
-			} catch (IOException e) {
-				throw new RuntimeException("Problem with jar file " + jarFile, e);
-			}
-		}
-		if (globalClasspaths == null) {
-			this.globalClasspaths = Collections.emptyList();
-		}
-		else {
-			this.globalClasspaths = Arrays.asList(globalClasspaths);
-		}
-	}
-
-	@Override
-	public JobExecutionResult execute(String jobName) throws ProgramInvocationException {
-		JobGraph jobGraph = getStreamGraph().getJobGraph(jobName);
-		transformations.clear();
-		return executeRemotely(jobGraph);
-	}
-
-	/**
-	 * Executes the remote job.
-	 * 
-	 * @param jobGraph
-	 *            jobGraph to execute
-	 * @return The result of the job execution, containing elapsed time and accumulators.
-	 */
-	private JobExecutionResult executeRemotely(JobGraph jobGraph) throws ProgramInvocationException {
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Running remotely at {}:{}", host, port);
-		}
-
-		for (URL jarFile : jarFiles) {
-			try {
-				jobGraph.addJar(new Path(jarFile.toURI()));
-			} catch (URISyntaxException e) {
-				throw new ProgramInvocationException("URL is invalid", e);
-			}
-		}
-
-		ClassLoader usercodeClassLoader = JobWithJars.buildUserCodeClassLoader(jarFiles, globalClasspaths,
-			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);
-
-		Client client;
-		try {
-			client = new Client(configuration);
-			client.setPrintStatusDuringExecution(getConfig().isSysoutLoggingEnabled());
-		}
-		catch (Exception e) {
-			throw new ProgramInvocationException("Cannot establish connection to JobManager: " + e.getMessage(), e);
-		}
-
-		try {
-			return client.runBlocking(jobGraph, usercodeClassLoader);
-		}
-		catch (ProgramInvocationException e) {
-			throw e;
-		}
-		catch (Exception e) {
-			String term = e.getMessage() == null ? "." : (": " + e.getMessage());
-			throw new ProgramInvocationException("The program execution failed" + term, e);
-		}
-		finally {
-			client.shutdown();
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "Remote Environment (" + this.host + ":" + this.port + " - parallelism = "
-				+ (getParallelism() == -1 ? "default" : getParallelism()) + ")";
-	}
-
-	/**
-	 * Gets the hostname of the master (JobManager), where the
-	 * program will be executed.
-	 *
-	 * @return The hostname of the master
-	 */
-	public String getHost() {
-		return host;
-	}
-
-	/**
-	 * Gets the port of the master (JobManager), where the
-	 * program will be executed.
-	 *
-	 * @return The port of the master
-	 */
-	public int getPort() {
-		return port;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
deleted file mode 100644
index b2a5435..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamContextEnvironment.java
+++ /dev/null
@@ -1,103 +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.environment;
-
-import java.net.URL;
-import java.util.List;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.JobSubmissionResult;
-import org.apache.flink.client.program.Client;
-import org.apache.flink.client.program.JobWithJars;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StreamContextEnvironment extends StreamExecutionEnvironment {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamContextEnvironment.class);
-
-	private final List<URL> jars;
-
-	private final List<URL> classpaths;
-	
-	private final Client client;
-
-	private final ClassLoader userCodeClassLoader;
-	
-	private final boolean wait;
-
-	protected StreamContextEnvironment(Client client, List<URL> jars, List<URL> classpaths, int parallelism,
-			boolean wait) {
-		this.client = client;
-		this.jars = jars;
-		this.classpaths = classpaths;
-		this.wait = wait;
-		
-		this.userCodeClassLoader = JobWithJars.buildUserCodeClassLoader(jars, classpaths,
-				getClass().getClassLoader());
-		
-		if (parallelism > 0) {
-			setParallelism(parallelism);
-		}
-		else {
-			// determine parallelism
-			setParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELISM_KEY,
-					ConfigConstants.DEFAULT_PARALLELISM));
-		}
-	}
-
-	@Override
-	public JobExecutionResult execute() throws Exception {
-		return execute(null);
-	}
-
-	@Override
-	public JobExecutionResult execute(String jobName) throws Exception {
-
-		JobGraph jobGraph;
-		if (jobName == null) {
-			jobGraph = this.getStreamGraph().getJobGraph();
-		} else {
-			jobGraph = this.getStreamGraph().getJobGraph(jobName);
-		}
-
-		transformations.clear();
-
-		// attach all necessary jar files to the JobGraph
-		for (URL file : jars) {
-			jobGraph.addJar(new Path(file.toURI()));
-		}
-
-		jobGraph.setClasspaths(classpaths);
-
-		// execute the programs
-		if (wait) {
-			return client.runBlocking(jobGraph, userCodeClassLoader);
-		} else {
-			JobSubmissionResult result = client.runDetached(jobGraph, userCodeClassLoader);
-			LOG.warn("Job was executed in detached mode, the results will be available on completion.");
-			return JobExecutionResult.fromJobSubmissionResult(result);
-		}
-	}
-}


[39/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumFunction.java
deleted file mode 100644
index e5fa4c3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumFunction.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.aggregation;
-
-import java.io.Serializable;
-
-public abstract class SumFunction implements Serializable{
-
-	private static final long serialVersionUID = 1L;
-
-	public abstract Object add(Object o1, Object o2);
-
-	public static SumFunction getForClass(Class<?> clazz) {
-
-		if (clazz == Integer.class) {
-			return new IntSum();
-		} else if (clazz == Long.class) {
-			return new LongSum();
-		} else if (clazz == Short.class) {
-			return new ShortSum();
-		} else if (clazz == Double.class) {
-			return new DoubleSum();
-		} else if (clazz == Float.class) {
-			return new FloatSum();
-		} else if (clazz == Byte.class) {
-			return new ByteSum();
-		} else {
-			throw new RuntimeException("DataStream cannot be summed because the class "
-					+ clazz.getSimpleName() + " does not support the + operator.");
-		}
-	}
-
-	public static class IntSum extends SumFunction {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Object add(Object value1, Object value2) {
-			return (Integer) value1 + (Integer) value2;
-		}
-	}
-
-	public static class LongSum extends SumFunction {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Object add(Object value1, Object value2) {
-			return (Long) value1 + (Long) value2;
-		}
-	}
-
-	public static class DoubleSum extends SumFunction {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Object add(Object value1, Object value2) {
-			return (Double) value1 + (Double) value2;
-		}
-	}
-
-	public static class ShortSum extends SumFunction {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Object add(Object value1, Object value2) {
-			return (short) ((Short) value1 + (Short) value2);
-		}
-	}
-
-	public static class FloatSum extends SumFunction {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Object add(Object value1, Object value2) {
-			return (Float) value1 + (Float) value2;
-		}
-	}
-
-	public static class ByteSum extends SumFunction {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Object add(Object value1, Object value2) {
-			return (byte) ((Byte) value1 + (Byte) value2);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoFlatMapFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoFlatMapFunction.java
deleted file mode 100644
index ae11cd9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoFlatMapFunction.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.co;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.util.Collector;
-
-/**
- * A CoFlatMapFunction represents a FlatMap transformation with two different
- * input types.
- *
- * @param <IN1>
- *            Type of the first input.
- * @param <IN2>
- *            Type of the second input.
- * @param <OUT>
- *            Output type.
- */
-public interface CoFlatMapFunction<IN1, IN2, OUT> extends Function, Serializable {
-
-	void flatMap1(IN1 value, Collector<OUT> out) throws Exception;
-
-	void flatMap2(IN2 value, Collector<OUT> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoMapFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoMapFunction.java
deleted file mode 100644
index a545282..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/CoMapFunction.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.co;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-
-/**
- * A CoMapFunction represents a Map transformation with two different input
- * types.
- *
- * @param <IN1>
- *            Type of the first input.
- * @param <IN2>
- *            Type of the second input.
- * @param <OUT>
- *            Output type.
- */
-public interface CoMapFunction<IN1, IN2, OUT> extends Function, Serializable {
-
-	OUT map1(IN1 value) throws Exception;
-
-	OUT map2(IN2 value) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoFlatMapFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoFlatMapFunction.java
deleted file mode 100644
index 6746140..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoFlatMapFunction.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.co;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * A RichCoFlatMapFunction represents a FlatMap transformation with two different input
- * types. In addition to that the user can use the features provided by the
- * {@link RichFunction} interface.
- *
- * @param <IN1>
- *            Type of the first input.
- * @param <IN2>
- *            Type of the second input.
- * @param <OUT>
- *            Output type.
- */
-public abstract class RichCoFlatMapFunction<IN1, IN2, OUT> extends AbstractRichFunction implements
-		CoFlatMapFunction<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoMapFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoMapFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoMapFunction.java
deleted file mode 100644
index e561408..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/co/RichCoMapFunction.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.co;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.api.common.functions.RichFunction;
-
-/**
- * A RichCoMapFunction represents a Map transformation with two different input
- * types. In addition to that the user can use the features provided by the
- * {@link RichFunction} interface.
- *
- * @param <IN1>
- *            Type of the first input.
- * @param <IN2>
- *            Type of the second input.
- * @param <OUT>
- *            Output type.
- */
-public abstract class RichCoMapFunction<IN1, IN2, OUT> extends AbstractRichFunction implements
-		CoMapFunction<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java
deleted file mode 100644
index 504bc39..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.io.CleanupWhenUnsuccessful;
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.configuration.Configuration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple implementation of the SinkFunction writing tuples in the specified
- * OutputFormat format. Tuples are collected to a list and written to the file
- * periodically. The target path and the overwrite mode are pre-packaged in
- * format.
- * 
- * @param <IN>
- *            Input type
- */
-public abstract class FileSinkFunction<IN> extends RichSinkFunction<IN> {
-	
-	private static final long serialVersionUID = 1L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(FileSinkFunction.class);
-	
-	protected ArrayList<IN> tupleList = new ArrayList<IN>();
-	protected volatile OutputFormat<IN> format;
-	protected volatile boolean cleanupCalled = false;
-	protected int indexInSubtaskGroup;
-	protected int currentNumberOfSubtasks;
-
-	public FileSinkFunction(OutputFormat<IN> format) {
-		this.format = format;
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		RuntimeContext context = getRuntimeContext();
-		format.configure(parameters);
-		indexInSubtaskGroup = context.getIndexOfThisSubtask();
-		currentNumberOfSubtasks = context.getNumberOfParallelSubtasks();
-		format.open(indexInSubtaskGroup, currentNumberOfSubtasks);
-	}
-
-	@Override
-	public void invoke(IN record) throws Exception {
-		tupleList.add(record);
-		if (updateCondition()) {
-			flush();
-		}
-	}
-
-	@Override
-	public void close() throws IOException {
-		if (!tupleList.isEmpty()) {
-			flush();
-		}
-		try {
-			format.close();
-		} catch (Exception ex) {
-			if (LOG.isErrorEnabled()) {
-				LOG.error("Error while writing element.", ex);
-			}
-			try {
-				if (!cleanupCalled && format instanceof CleanupWhenUnsuccessful) {
-					cleanupCalled = true;
-					((CleanupWhenUnsuccessful) format).tryCleanupOnError();
-				}
-			} catch (Throwable t) {
-				LOG.error("Cleanup on error failed.", t);
-			}
-		}
-	}
-
-	protected void flush() {
-		try {
-			for (IN rec : tupleList) {
-				format.writeRecord(rec);
-			}
-		} catch (Exception ex) {
-			try {
-				if (LOG.isErrorEnabled()) {
-					LOG.error("Error while writing element.", ex);
-				}
-				if (!cleanupCalled && format instanceof CleanupWhenUnsuccessful) {
-					cleanupCalled = true;
-					((CleanupWhenUnsuccessful) format).tryCleanupOnError();
-				}
-			} catch (Throwable t) {
-				LOG.error("Cleanup on error failed.", t);
-			}
-			throw new RuntimeException(ex);
-		}
-		resetParameters();
-	}
-
-	/**
-	 * Condition for writing the contents of tupleList and clearing it.
-	 * 
-	 * @return value of the updating condition
-	 */
-	protected abstract boolean updateCondition();
-
-	/**
-	 * Statements to be executed after writing a batch goes here.
-	 */
-	protected abstract void resetParameters();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunctionByMillis.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunctionByMillis.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunctionByMillis.java
deleted file mode 100644
index 86bbb53..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunctionByMillis.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import org.apache.flink.api.common.io.OutputFormat;
-
-/**
- * Implementation of FileSinkFunction. Writes tuples to file in every millis
- * milliseconds.
- * 
- * @param <IN>
- *            Input type
- */
-public class FileSinkFunctionByMillis<IN> extends FileSinkFunction<IN> {
-	private static final long serialVersionUID = 1L;
-
-	private final long millis;
-	private long lastTime;
-
-	public FileSinkFunctionByMillis(OutputFormat<IN> format, long millis) {
-		super(format);
-		this.millis = millis;
-		lastTime = System.currentTimeMillis();
-	}
-
-	/**
-	 * Condition for writing the contents of tupleList and clearing it.
-	 * 
-	 * @return value of the updating condition
-	 */
-	@Override
-	protected boolean updateCondition() {
-		return System.currentTimeMillis() - lastTime >= millis;
-	}
-
-	/**
-	 * Statements to be executed after writing a batch goes here.
-	 */
-	@Override
-	protected void resetParameters() {
-		tupleList.clear();
-		lastTime = System.currentTimeMillis();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java
deleted file mode 100644
index 93a91cd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.PrintStream;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-
-/**
- * Implementation of the SinkFunction writing every tuple to the standard
- * output or standard error stream.
- * 
- * @param <IN>
- *            Input record type
- */
-public class PrintSinkFunction<IN> extends RichSinkFunction<IN> {
-	private static final long serialVersionUID = 1L;
-
-	private static final boolean STD_OUT = false;
-	private static final boolean STD_ERR = true;
-	
-	private boolean target; 
-	private transient PrintStream stream;
-	private transient String prefix;
-	
-	/**
-	 * Instantiates a print sink function that prints to standard out.
-	 */
-	public PrintSinkFunction() {}
-	
-	/**
-	 * Instantiates a print sink function that prints to standard out.
-	 * 
-	 * @param stdErr True, if the format should print to standard error instead of standard out.
-	 */
-	public PrintSinkFunction(boolean stdErr) {
-		target = stdErr;
-	}
-
-	public void setTargetToStandardOut() {
-		target = STD_OUT;
-	}
-	
-	public void setTargetToStandardErr() {
-		target = STD_ERR;
-	}
-	
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		StreamingRuntimeContext context = (StreamingRuntimeContext) getRuntimeContext();
-		// get the target stream
-		stream = target == STD_OUT ? System.out : System.err;
-		
-		// set the prefix if we have a >1 parallelism
-		prefix = (context.getNumberOfParallelSubtasks() > 1) ? 
-				((context.getIndexOfThisSubtask() + 1) + "> ") : null;
-	}
-
-	@Override
-	public void invoke(IN record) {
-		if (prefix != null) {
-			stream.println(prefix + record.toString());
-		}
-		else {
-			stream.println(record.toString());
-		}
-	}
-	
-	@Override
-	public void close() {
-		this.stream = null;
-		this.prefix = null;
-	}
-	
-	@Override
-	public String toString() {
-		return "Print to " + (target == STD_OUT ? "System.out" : "System.err");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/RichSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/RichSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/RichSinkFunction.java
deleted file mode 100644
index 7853758..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/RichSinkFunction.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-
-public abstract class RichSinkFunction<IN> extends AbstractRichFunction implements SinkFunction<IN> {
-
-	private static final long serialVersionUID = 1L;
-
-	public abstract void invoke(IN value) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java
deleted file mode 100644
index 21308ed..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SinkFunction.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-
-/**
- * Interface for implementing user defined sink functionality.
- *
- * @param <IN> Input type parameter.
- */
-public interface SinkFunction<IN> extends Function, Serializable {
-
-	/**
-	 * Function for standard sink behaviour. This function is called for every record.
-	 *
-	 * @param value The input record.
-	 * @throws Exception
-	 */
-	void invoke(IN value) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java
deleted file mode 100644
index 1356263..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *	http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.Socket;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.util.SerializableObject;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Socket client that acts as a streaming sink. The data is sent to a Socket as a byte array.
- * <p>
- * The sink can be set to retry message sends after the sending failed.
- * <p>
- * The sink can be set to 'autoflush', in which case the socket stream is flushed after every message. This
- * significantly reduced throughput, but also decreases message latency.
- *
- * @param <IN> data to be written into the Socket.
- */
-public class SocketClientSink<IN> extends RichSinkFunction<IN> {
-
-	private static final long serialVersionUID = 1L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(SocketClientSink.class);
-
-	private static final int CONNECTION_RETRY_DELAY = 500;
-	
-	
-	private final SerializableObject lock = new SerializableObject();
-	private final SerializationSchema<IN, byte[]> schema;
-	private final String hostName;
-	private final int port;
-	private final int maxNumRetries;
-	private final boolean autoFlush;
-	
-	private transient Socket client;
-	private transient OutputStream outputStream;
-	
-	private int retries;
-
-	private volatile boolean isRunning = true;
-	
-	/**
-	 * Creates a new SocketClientSink. The sink will not attempt to retry connections upon failure
-	 * and will not auto-flush the stream.
-	 *
-	 * @param hostName Hostname of the server to connect to.
-	 * @param port Port of the server.
-	 * @param schema Schema used to serialize the data into bytes.
-	 */
-	public SocketClientSink(String hostName, int port, SerializationSchema<IN, byte[]> schema) {
-		this(hostName, port, schema, 0);
-	}
-
-	/**
-	 * Creates a new SocketClientSink that retries connections upon failure up to a given number of times.
-	 * A value of -1 for the number of retries will cause the system to retry an infinite number of times.
-	 * The sink will not auto-flush the stream.
-	 *
-	 * @param hostName Hostname of the server to connect to.
-	 * @param port Port of the server.
-	 * @param schema Schema used to serialize the data into bytes.
-	 * @param maxNumRetries The maximum number of retries after a message send failed.
-	 */
-	public SocketClientSink(String hostName, int port, SerializationSchema<IN, byte[]> schema, int maxNumRetries) {
-		this(hostName, port, schema, maxNumRetries, false);
-	}
-
-	/**
-	 * Creates a new SocketClientSink that retries connections upon failure up to a given number of times.
-	 * A value of -1 for the number of retries will cause the system to retry an infinite number of times.
-	 *
-	 * @param hostName Hostname of the server to connect to.
-	 * @param port Port of the server.
-	 * @param schema Schema used to serialize the data into bytes.
-	 * @param maxNumRetries The maximum number of retries after a message send failed.
-	 * @param autoflush Flag to indicate whether the socket stream should be flushed after each message.
-	 */
-	public SocketClientSink(String hostName, int port, SerializationSchema<IN, byte[]> schema,
-							int maxNumRetries, boolean autoflush)
-	{
-		checkArgument(port > 0 && port < 65536, "port is out of range");
-		checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
-
-		this.hostName = checkNotNull(hostName, "hostname must not be null");
-		this.port = port;
-		this.schema = checkNotNull(schema);
-		this.maxNumRetries = maxNumRetries;
-		this.autoFlush = autoflush;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Life cycle
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Initialize the connection with the Socket in the server.
-	 * @param parameters Configuration.
-	 */
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		try {
-			synchronized (lock) {
-				createConnection();
-			}
-		}
-		catch (IOException e) {
-			throw new IOException("Cannot connect to socket server at " + hostName + ":" + port, e);
-		}
-	}
-	
-	
-	/**
-	 * Called when new data arrives to the sink, and forwards it to Socket.
-	 *
-	 * @param value The value to write to the socket.
-	 */
-	@Override
-	public void invoke(IN value) throws Exception {
-		byte[] msg = schema.serialize(value);
-
-		try {
-			outputStream.write(msg);
-			if (autoFlush) {
-				outputStream.flush();
-			}
-		}
-		catch (IOException e) {
-			// if no re-tries are enable, fail immediately
-			if (maxNumRetries == 0) {
-				throw new IOException("Failed to send message '" + value + "' to socket server at "
-						+ hostName + ":" + port + ". Connection re-tries are not enabled.", e);
-			}
-			
-			LOG.error("Failed to send message '" + value + "' to socket server at " + hostName + ":" + port + 
-					". Trying to reconnect..." , e);
-			
-			// do the retries in locked scope, to guard against concurrent close() calls
-			// note that the first re-try comes immediately, without a wait!
-		
-			synchronized (lock) {
-				IOException lastException = null;
-				retries = 0;
-				
-				while (isRunning && (maxNumRetries < 0 || retries < maxNumRetries)) {
-					
-					// first, clean up the old resources
-					try {
-						if (outputStream != null) {
-							outputStream.close();
-						}
-					}
-					catch (IOException ee) {
-						LOG.error("Could not close output stream from failed write attempt", ee);
-					}
-					try {
-						if (client != null) {
-							client.close();
-						}
-					}
-					catch (IOException ee) {
-						LOG.error("Could not close socket from failed write attempt", ee);
-					}
-					
-					// try again
-					retries++;
-					
-					try {
-						// initialize a new connection
-						createConnection();
-						
-						// re-try the write
-						outputStream.write(msg);
-						
-						// success!
-						return;
-					}
-					catch (IOException ee) {
-						lastException = ee;
-						LOG.error("Re-connect to socket server and send message failed. Retry time(s): " + retries, ee);
-					}
-
-					// wait before re-attempting to connect
-					lock.wait(CONNECTION_RETRY_DELAY);
-				}
-				
-				// throw an exception if the task is still running, otherwise simply leave the method
-				if (isRunning) {
-					throw new IOException("Failed to send message '" + value + "' to socket server at "
-							+ hostName + ":" + port + ". Failed after " + retries + " retries.", lastException);
-				}
-			}
-		}
-	}
-	
-	/**
-	 * Closes the connection with the Socket server.
-	 */
-	@Override
-	public void close() throws Exception {
-		// flag this as not running any more
-		isRunning = false;
-		
-		// clean up in locked scope, so there is no concurrent change to the stream and client
-		synchronized (lock) {
-			// we notify first (this statement cannot fail). The notified thread will not continue
-			// anyways before it can re-acquire the lock
-			lock.notifyAll();
-			
-			try {
-				if (outputStream != null) {
-					outputStream.close();
-				}
-			}
-			finally {
-				if (client != null) {
-					client.close();
-				}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-	
-	private void createConnection() throws IOException {
-		client = new Socket(hostName, port);
-		client.setKeepAlive(true);
-		client.setTcpNoDelay(true);
-		
-		outputStream = client.getOutputStream();
-	}
-	
-	// ------------------------------------------------------------------------
-	//  For testing
-	// ------------------------------------------------------------------------
-	
-	int getCurrentNumberOfRetries() {
-		synchronized (lock) {
-			return retries;
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormat.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormat.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormat.java
deleted file mode 100644
index 019d35f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormat.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-
-/**
- * Abstract class for formatting the output of the writeAsText and writeAsCsv
- * functions.
- *
- * @param <IN>
- *            Input tuple type
- */
-public abstract class WriteFormat<IN> implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Writes the contents of tupleList to the file specified by path.
-	 * 
-	 * @param path
-	 *            is the path to the location where the tuples are written
-	 * @param tupleList
-	 *            is the list of tuples to be written
-	 */
-	protected abstract void write(String path, ArrayList<IN> tupleList);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsCsv.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsCsv.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsCsv.java
deleted file mode 100644
index bfae653..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsCsv.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.BufferedWriter;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-
-/**
- * Writes tuples in csv format.
- * 
- * @param <IN>
- *            Input tuple type
- */
-public class WriteFormatAsCsv<IN> extends WriteFormat<IN> {
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	protected void write(String path, ArrayList<IN> tupleList) {
-		try {
-			PrintWriter outStream = new PrintWriter(new BufferedWriter(new FileWriter(path, true)));
-			for (IN tupleToWrite : tupleList) {
-				String strTuple = tupleToWrite.toString();
-				outStream.println(strTuple.substring(1, strTuple.length() - 1));
-			}
-			outStream.close();
-		} catch (IOException e) {
-			throw new RuntimeException("Exception occured while writing file " + path, e);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsText.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsText.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsText.java
deleted file mode 100644
index 03fcb5c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteFormatAsText.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.BufferedWriter;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-
-/**
- * Writes tuples in text format.
- *
- * @param <IN>
- *            Input tuple type
- */
-public class WriteFormatAsText<IN> extends WriteFormat<IN> {
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public void write(String path, ArrayList<IN> tupleList) {
-		try {
-			PrintWriter outStream = new PrintWriter(new BufferedWriter(new FileWriter(path, true)));
-			for (IN tupleToWrite : tupleList) {
-				outStream.println(tupleToWrite);
-			}
-			outStream.close();
-		} catch (IOException e) {
-			throw new RuntimeException("Exception occured while writing file " + path, e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunction.java
deleted file mode 100644
index 27c352f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunction.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import java.io.FileNotFoundException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-
-/**
- * Simple implementation of the SinkFunction writing tuples as simple text to
- * the file specified by path. Tuples are collected to a list and written to the
- * file periodically. The file specified by path is created if it does not
- * exist, cleared if it exists before the writing.
- * 
- * @param <IN>
- *            Input tuple type
- */
-public abstract class WriteSinkFunction<IN> implements SinkFunction<IN> {
-	private static final long serialVersionUID = 1L;
-
-	protected final String path;
-	protected ArrayList<IN> tupleList = new ArrayList<IN>();
-	protected WriteFormat<IN> format;
-
-	public WriteSinkFunction(String path, WriteFormat<IN> format) {
-		this.path = path;
-		this.format = format;
-		cleanFile(path);
-	}
-
-	/**
-	 * Creates target file if it does not exist, cleans it if it exists.
-	 * 
-	 * @param path
-	 *            is the path to the location where the tuples are written
-	 */
-	protected void cleanFile(String path) {
-		try {
-			PrintWriter writer;
-			writer = new PrintWriter(path);
-			writer.print("");
-			writer.close();
-		} catch (FileNotFoundException e) {
-			throw new RuntimeException("An error occurred while cleaning the file: " + e.getMessage(), e);
-		}
-	}
-
-	/**
-	 * Condition for writing the contents of tupleList and clearing it.
-	 * 
-	 * @return value of the updating condition
-	 */
-	protected abstract boolean updateCondition();
-
-	/**
-	 * Statements to be executed after writing a batch goes here.
-	 */
-	protected abstract void resetParameters();
-
-	/**
-	 * Implementation of the invoke method of the SinkFunction class. Collects
-	 * the incoming tuples in tupleList and appends the list to the end of the
-	 * target file if updateCondition() is true or the current tuple is the
-	 * endTuple.
-	 */
-	@Override
-	public void invoke(IN tuple) {
-
-		tupleList.add(tuple);
-		if (updateCondition()) {
-			format.write(path, tupleList);
-			resetParameters();
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunctionByMillis.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunctionByMillis.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunctionByMillis.java
deleted file mode 100644
index 0364174..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/WriteSinkFunctionByMillis.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-/**
- * Implementation of WriteSinkFunction. Writes tuples to file in every millis
- * milliseconds.
- * 
- * @param <IN>
- *            Input tuple type
- */
-public class WriteSinkFunctionByMillis<IN> extends WriteSinkFunction<IN> {
-	private static final long serialVersionUID = 1L;
-
-	private final long millis;
-	private long lastTime;
-
-	public WriteSinkFunctionByMillis(String path, WriteFormat<IN> format, long millis) {
-		super(path, format);
-		this.millis = millis;
-		lastTime = System.currentTimeMillis();
-	}
-
-	@Override
-	protected boolean updateCondition() {
-		return System.currentTimeMillis() - lastTime >= millis;
-	}
-
-	@Override
-	protected void resetParameters() {
-		tupleList.clear();
-		lastTime = System.currentTimeMillis();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ConnectorSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ConnectorSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ConnectorSource.java
deleted file mode 100644
index 0d107f6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ConnectorSource.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-public abstract class ConnectorSource<OUT> extends RichParallelSourceFunction<OUT> implements ResultTypeQueryable<OUT> {
-
-	private static final long serialVersionUID = 1L;
-	
-	protected DeserializationSchema<OUT> schema;
-
-	public ConnectorSource(DeserializationSchema<OUT> schema) {
-		this.schema = schema;
-	}
-
-	@Override
-	public TypeInformation<OUT> getProducedType() {
-		return schema.getProducedType();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/EventTimeSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/EventTimeSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/EventTimeSourceFunction.java
deleted file mode 100644
index ab380d7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/EventTimeSourceFunction.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.functions.source;
-
-
-/**
- * A marker interface that must be implemented by {@link SourceFunction}s that emit elements with
- * timestamps. The {@link SourceFunction} can extract the timestamp from the data and attach it to
- * the element upon emission.
- *
- * <p>
- * Event-time sources must manually emit
- * {@link org.apache.flink.streaming.api.watermark.Watermark watermarks} to keep track of progress.
- * Automatic emission of watermarks will be suppressed if a source implements this interface.
- *
- * <p>
- * Elements must be emitted using
- * {@link SourceFunction.SourceContext#collectWithTimestamp(Object, long)}
- * and watermarks can be emitted using
- * {@link SourceFunction.SourceContext#emitWatermark(org.apache.flink.streaming.api.watermark.Watermark)}.
- *
- * @param <T> Type of the elements emitted by this source.
- */
-public interface EventTimeSourceFunction<T> extends SourceFunction<T> { }

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunction.java
deleted file mode 100644
index a217923..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunction.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class FileMonitoringFunction implements SourceFunction<Tuple3<String, Long, Long>> {
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(FileMonitoringFunction.class);
-
-	public enum WatchType {
-		ONLY_NEW_FILES, // Only new files will be processed.
-		REPROCESS_WITH_APPENDED, // When some files are appended, all contents
-									// of the files will be processed.
-		PROCESS_ONLY_APPENDED // When some files are appended, only appended
-								// contents will be processed.
-	}
-
-	private String path;
-	private long interval;
-	private WatchType watchType;
-
-	private Map<String, Long> offsetOfFiles;
-	private Map<String, Long> modificationTimes;
-
-	private volatile boolean isRunning = true;
-
-	public FileMonitoringFunction(String path, long interval, WatchType watchType) {
-		this.path = path;
-		this.interval = interval;
-		this.watchType = watchType;
-		this.modificationTimes = new HashMap<String, Long>();
-		this.offsetOfFiles = new HashMap<String, Long>();
-	}
-
-	@Override
-	public void run(SourceContext<Tuple3<String, Long, Long>> ctx) throws Exception {
-		FileSystem fileSystem = FileSystem.get(new URI(path));
-
-		while (isRunning) {
-			List<String> files = listNewFiles(fileSystem);
-			for (String filePath : files) {
-				if (watchType == WatchType.ONLY_NEW_FILES
-						|| watchType == WatchType.REPROCESS_WITH_APPENDED) {
-					ctx.collect(new Tuple3<String, Long, Long>(filePath, 0L, -1L));
-					offsetOfFiles.put(filePath, -1L);
-				} else if (watchType == WatchType.PROCESS_ONLY_APPENDED) {
-					long offset = 0;
-					long fileSize = fileSystem.getFileStatus(new Path(filePath)).getLen();
-					if (offsetOfFiles.containsKey(filePath)) {
-						offset = offsetOfFiles.get(filePath);
-					}
-
-					ctx.collect(new Tuple3<String, Long, Long>(filePath, offset, fileSize));
-					offsetOfFiles.put(filePath, fileSize);
-
-					LOG.info("File processed: {}, {}, {}", filePath, offset, fileSize);
-				}
-			}
-
-			Thread.sleep(interval);
-		}
-	}
-
-	private List<String> listNewFiles(FileSystem fileSystem) throws IOException {
-		List<String> files = new ArrayList<String>();
-
-		FileStatus[] statuses = fileSystem.listStatus(new Path(path));
-
-		if (statuses == null) {
-			LOG.warn("Path does not exist: {}", path);
-		} else {
-			for (FileStatus status : statuses) {
-				Path filePath = status.getPath();
-				String fileName = filePath.getName();
-				long modificationTime = status.getModificationTime();
-
-				if (!isFiltered(fileName, modificationTime)) {
-					files.add(filePath.toString());
-					modificationTimes.put(fileName, modificationTime);
-				}
-			}
-		}
-
-		return files;
-	}
-
-	private boolean isFiltered(String fileName, long modificationTime) {
-
-		if ((watchType == WatchType.ONLY_NEW_FILES && modificationTimes.containsKey(fileName))
-				|| fileName.startsWith(".") || fileName.contains("_COPYING_")) {
-			return true;
-		} else {
-			Long lastModification = modificationTimes.get(fileName);
-			return lastModification != null && lastModification >= modificationTime;
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileReadFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileReadFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileReadFunction.java
deleted file mode 100644
index 4f859e8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileReadFunction.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
-import java.net.URI;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.Collector;
-
-public class FileReadFunction implements FlatMapFunction<Tuple3<String, Long, Long>, String> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public void flatMap(Tuple3<String, Long, Long> value, Collector<String> out) throws Exception {
-		FSDataInputStream stream = FileSystem.get(new URI(value.f0)).open(new Path(value.f0));
-		stream.seek(value.f1);
-
-		BufferedReader reader = new BufferedReader(new InputStreamReader(stream));
-		String line;
-
-		try {
-			while ((line = reader.readLine()) != null && (value.f2 == -1L || stream.getPos() <= value.f2)) {
-				out.collect(line);
-			}
-		} finally {
-			reader.close();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
deleted file mode 100644
index cc3925c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-public class FileSourceFunction<OUT> extends RichParallelSourceFunction<OUT> {
-	private static final long serialVersionUID = 1L;
-
-	private TypeInformation<OUT> typeInfo;
-	private transient TypeSerializer<OUT> serializer;
-
-	private InputFormat<OUT, InputSplit> format;
-
-	private transient InputSplitProvider provider;
-	private transient Iterator<InputSplit> splitIterator;
-
-	private volatile boolean isRunning = true;
-
-	@SuppressWarnings("unchecked")
-	public FileSourceFunction(InputFormat<OUT, ?> format, TypeInformation<OUT> typeInfo) {
-		this.format = (InputFormat<OUT, InputSplit>) format;
-		this.typeInfo = typeInfo;
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public void open(Configuration parameters) throws Exception {
-		StreamingRuntimeContext context = (StreamingRuntimeContext) getRuntimeContext();
-		this.provider = context.getInputSplitProvider();
-		
-		format.configure(parameters);
-		serializer = typeInfo.createSerializer(getRuntimeContext().getExecutionConfig());
-
-		splitIterator = getInputSplits();
-		if (splitIterator.hasNext()) {
-			format.open(splitIterator.next());
-		}
-		isRunning = true;
-	}
-
-	@Override
-	public void close() throws Exception {
-		format.close();
-	}
-
-	private Iterator<InputSplit> getInputSplits() {
-
-		return new Iterator<InputSplit>() {
-
-			private InputSplit nextSplit;
-
-			private boolean exhausted;
-
-			@Override
-			public boolean hasNext() {
-				if (exhausted) {
-					return false;
-				}
-
-				if (nextSplit != null) {
-					return true;
-				}
-
-				InputSplit split = provider.getNextInputSplit();
-
-				if (split != null) {
-					this.nextSplit = split;
-					return true;
-				} else {
-					exhausted = true;
-					return false;
-				}
-			}
-
-			@Override
-			public InputSplit next() {
-				if (this.nextSplit == null && !hasNext()) {
-					throw new NoSuchElementException();
-				}
-
-				final InputSplit tmp = this.nextSplit;
-				this.nextSplit = null;
-				return tmp;
-			}
-
-			@Override
-			public void remove() {
-				throw new UnsupportedOperationException();
-			}
-		};
-	}
-
-	@Override
-	public void run(SourceContext<OUT> ctx) throws Exception {
-		while (isRunning) {
-			OUT nextElement = serializer.createInstance();
-			nextElement =  format.nextRecord(nextElement);
-			if (nextElement == null && splitIterator.hasNext()) {
-				format.open(splitIterator.next());
-				continue;
-			} else if (nextElement == null) {
-				break;
-			}
-			ctx.collect(nextElement);
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-	}
-
-
-	/**
-	 * Returns the {@code InputFormat}. This is only needed because we need to set the input
-	 * split assigner on the {@code StreamGraph}.
-	 */
-	public InputFormat<OUT, InputSplit> getFormat() {
-		return format;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java
deleted file mode 100644
index af47f59..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromElementsFunction.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-
-/**
- * A stream source function that returns a sequence of elements.
- * 
- * <p>Upon construction, this source function serializes the elements using Flink's type information.
- * That way, any object transport using Java serialization will not be affected by the serializability
- * of the elements.</p>
- * 
- * @param <T> The type of elements returned by this function.
- */
-public class FromElementsFunction<T> implements SourceFunction<T>, CheckpointedAsynchronously<Integer> {
-	
-	private static final long serialVersionUID = 1L;
-
-	/** The (de)serializer to be used for the data elements */
-	private final TypeSerializer<T> serializer;
-	
-	/** The actual data elements, in serialized form */
-	private final byte[] elementsSerialized;
-	
-	/** The number of serialized elements */
-	private final int numElements;
-
-	/** The number of elements emitted already */
-	private volatile int numElementsEmitted;
-
-	/** The number of elements to skip initially */
-	private volatile int numElementsToSkip;
-	
-	/** Flag to make the source cancelable */
-	private volatile boolean isRunning = true;
-
-	
-	public FromElementsFunction(TypeSerializer<T> serializer, T... elements) throws IOException {
-		this(serializer, Arrays.asList(elements));
-	}
-	
-	public FromElementsFunction(TypeSerializer<T> serializer, Iterable<T> elements) throws IOException {
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		OutputViewDataOutputStreamWrapper wrapper = new OutputViewDataOutputStreamWrapper(new DataOutputStream(baos));
-
-		int count = 0;
-		try {
-			for (T element : elements) {
-				serializer.serialize(element, wrapper);
-				count++;
-			}
-		}
-		catch (Exception e) {
-			throw new IOException("Serializing the source elements failed: " + e.getMessage(), e);
-		}
-
-		this.serializer = serializer;
-		this.elementsSerialized = baos.toByteArray();
-		this.numElements = count;
-	}
-
-	@Override
-	public void run(SourceContext<T> ctx) throws Exception {
-		ByteArrayInputStream bais = new ByteArrayInputStream(elementsSerialized);
-		final DataInputView input = new InputViewDataInputStreamWrapper(new DataInputStream(bais));
-		
-		// if we are restored from a checkpoint and need to skip elements, skip them now.
-		int toSkip = numElementsToSkip;
-		if (toSkip > 0) {
-			try {
-				while (toSkip > 0) {
-					serializer.deserialize(input);
-					toSkip--;
-				}
-			}
-			catch (Exception e) {
-				throw new IOException("Failed to deserialize an element from the source. " +
-						"If you are using user-defined serialization (Value and Writable types), check the " +
-						"serialization functions.\nSerializer is " + serializer);
-			}
-			
-			this.numElementsEmitted = this.numElementsToSkip;
-		}
-		
-		final Object lock = ctx.getCheckpointLock();
-		
-		while (isRunning && numElementsEmitted < numElements) {
-			T next;
-			try {
-				next = serializer.deserialize(input);
-			}
-			catch (Exception e) {
-				throw new IOException("Failed to deserialize an element from the source. " +
-						"If you are using user-defined serialization (Value and Writable types), check the " +
-						"serialization functions.\nSerializer is " + serializer);
-			}
-			
-			synchronized (lock) {
-				ctx.collect(next);
-				numElementsEmitted++;
-			}
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-	}
-
-
-	/**
-	 * Gets the number of elements produced in total by this function.
-	 * 
-	 * @return The number of elements produced in total.
-	 */
-	public int getNumElements() {
-		return numElements;
-	}
-
-	/**
-	 * Gets the number of elements emitted so far.
-	 * 
-	 * @return The number of elements emitted so far.
-	 */
-	public int getNumElementsEmitted() {
-		return numElementsEmitted;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpointing
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-		return this.numElementsEmitted;
-	}
-
-	@Override
-	public void restoreState(Integer state) {
-		this.numElementsToSkip = state;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Verifies that all elements in the collection are non-null, and are of the given class, or
-	 * a subclass thereof.
-	 * 
-	 * @param elements The collection to check.
-	 * @param viewedAs The class to which the elements must be assignable to.
-	 * 
-	 * @param <OUT> The generic type of the collection to be checked.
-	 */
-	public static <OUT> void checkCollection(Collection<OUT> elements, Class<OUT> viewedAs) {
-		for (OUT elem : elements) {
-			if (elem == null) {
-				throw new IllegalArgumentException("The collection contains a null element");
-			}
-
-			if (!viewedAs.isAssignableFrom(elem.getClass())) {
-				throw new IllegalArgumentException("The elements in the collection are not all subclasses of " +
-						viewedAs.getCanonicalName());
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromIteratorFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromIteratorFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromIteratorFunction.java
deleted file mode 100644
index 655710e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromIteratorFunction.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import java.util.Iterator;
-
-public class FromIteratorFunction<T> implements SourceFunction<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final Iterator<T> iterator;
-
-	private volatile boolean isRunning = true;
-
-	public FromIteratorFunction(Iterator<T> iterator) {
-		this.iterator = iterator;
-	}
-
-	@Override
-	public void run(SourceContext<T> ctx) throws Exception {
-		while (isRunning && iterator.hasNext()) {
-			ctx.collect(iterator.next());
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.java
deleted file mode 100644
index bc78e4d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.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.functions.source;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.SplittableIterator;
-
-import java.util.Iterator;
-
-public class FromSplittableIteratorFunction<T> extends RichParallelSourceFunction<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	private SplittableIterator<T> fullIterator;
-
-	private transient Iterator<T> iterator;
-
-	private volatile boolean isRunning = true;
-
-	public FromSplittableIteratorFunction(SplittableIterator<T> iterator) {
-		this.fullIterator = iterator;
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		int numberOfSubTasks = getRuntimeContext().getNumberOfParallelSubtasks();
-		int indexofThisSubTask = getRuntimeContext().getIndexOfThisSubtask();
-		iterator = fullIterator.split(numberOfSubTasks)[indexofThisSubTask];
-		isRunning = true;
-	}
-
-	@Override
-	public void run(SourceContext<T> ctx) throws Exception {
-		while (isRunning && iterator.hasNext()) {
-			ctx.collect(iterator.next());
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java
deleted file mode 100644
index 3ac63af..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledingSourceBase.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-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.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.runtime.state.SerializedCheckpointData;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * Abstract base class for data sources that receive elements from a message queue and
- * acknowledge them back by IDs.
- * <p>
- * The mechanism for this source assumes that messages are identified by a unique ID.
- * When messages are taken from the message queue, the message must not be dropped immediately,
- * but must be retained until acknowledged. Messages that are not acknowledged within a certain
- * time interval will be served again (to a different connection, established by the recovered source).
- * <p>
- * Note that this source can give no guarantees about message order in teh case of failures,
- * because messages that were retrieved but not yet acknowledged will be returned later again, after
- * a set of messages that was not retrieved before the failure.
- * <p>
- * Internally, this source gathers the IDs of elements it emits. Per checkpoint, the IDs are stored and
- * acknowledged when the checkpoint is complete. That way, no message is acknowledged unless it is certain
- * that it has been successfully processed throughout the topology and the updates to any state caused by
- * that message are persistent.
- * <p>
- * All messages that are emitted and successfully processed by the streaming program will eventually be
- * acknowledged. In corner cases, the source may acknowledge certain IDs multiple times, if a
- * failure occurs while acknowledging.
- * <p>
- * A typical way to use this base in a source function is by implementing a run() method as follows:
- * <pre>{@code
- * public void run(SourceContext<Type> ctx) throws Exception {
- *     while (running) {
- *         Message msg = queue.retrieve();
- *         synchronized (ctx.getCheckpointLock()) {
- *             ctx.collect(msg.getMessageData());
- *             addId(msg.getMessageId());
- *         }
- *     }
- * }
- * }</pre>
- * 
- * @param <Type> The type of the messages created by the source.
- * @param <Id> The type of the IDs that are used for acknowledging elements.
- */
-public abstract class MessageAcknowledingSourceBase<Type, Id> extends RichSourceFunction<Type> 
-	implements Checkpointed<SerializedCheckpointData[]>, CheckpointNotifier {
-	
-	private static final long serialVersionUID = -8689291992192955579L;
-	
-	/** Serializer used to serialize the IDs for checkpoints */
-	private final TypeSerializer<Id> idSerializer;
-	
-	/** The list gathering the IDs of messages emitted during the current checkpoint */
-	private transient List<Id> idsForCurrentCheckpoint;
-
-	/** The list with IDs from checkpoints that were triggered, but not yet completed or notified of completion */
-	private transient ArrayDeque<Tuple2<Long, List<Id>>> pendingCheckpoints;
-
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new MessageAcknowledingSourceBase for IDs of teh given type.
-	 * 
-	 * @param idClass The class of the message ID type, used to create a serializer for the message IDs.
-	 */
-	protected MessageAcknowledingSourceBase(Class<Id> idClass) {
-		this(TypeExtractor.getForClass(idClass));
-	}
-
-	/**
-	 * Creates a new MessageAcknowledingSourceBase for IDs of teh given type.
-	 * 
-	 * @param idTypeInfo The type information of the message ID type, used to create a serializer for the message IDs.
-	 */
-	protected MessageAcknowledingSourceBase(TypeInformation<Id> idTypeInfo) {
-		this.idSerializer = idTypeInfo.createSerializer(new ExecutionConfig());
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		idsForCurrentCheckpoint = new ArrayList<>(64);
-		pendingCheckpoints = new ArrayDeque<>();
-	}
-
-	@Override
-	public void close() throws Exception {
-		idsForCurrentCheckpoint.clear();
-		pendingCheckpoints.clear();
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  ID Checkpointing
-	// ------------------------------------------------------------------------
-
-	/**
-	 * This method must be implemented to acknowledge the given set of IDs back to the message queue.
-	 * @param ids The list od IDs to acknowledge.
-	 */
-	protected abstract void acknowledgeIDs(List<Id> ids);
-
-	/**
-	 * Adds an ID to be stored with the current checkpoint.
-	 * @param id The ID to add.
-	 */
-	protected void addId(Id id) {
-		idsForCurrentCheckpoint.add(id);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpointing the data
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public SerializedCheckpointData[] snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		pendingCheckpoints.addLast(new Tuple2<Long, List<Id>>(checkpointId, idsForCurrentCheckpoint));
-		idsForCurrentCheckpoint = new ArrayList<>(64);
-		
-		return SerializedCheckpointData.fromDeque(pendingCheckpoints, idSerializer);
-	}
-
-	@Override
-	public void restoreState(SerializedCheckpointData[] state) throws Exception {
-		pendingCheckpoints = SerializedCheckpointData.toDeque(state, idSerializer);
-	}
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		for (Iterator<Tuple2<Long, List<Id>>> iter = pendingCheckpoints.iterator(); iter.hasNext();) {
-			Tuple2<Long, List<Id>> checkpoint = iter.next();
-			long id = checkpoint.f0;
-			
-			if (id <= checkpointId) {
-				acknowledgeIDs(checkpoint.f1);
-				iter.remove();
-			}
-			else {
-				break;
-			}
-		}
-	}
-}


[09/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
new file mode 100644
index 0000000..e9a5728
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -0,0 +1,1124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+import kafka.admin.AdminUtils;
+import kafka.api.PartitionMetadata;
+import kafka.consumer.Consumer;
+import kafka.consumer.ConsumerConfig;
+import kafka.consumer.ConsumerIterator;
+import kafka.consumer.KafkaStream;
+import kafka.javaapi.consumer.ConsumerConnector;
+import kafka.message.MessageAndMetadata;
+import kafka.server.KafkaServer;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.commons.collections.map.LinkedMap;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+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.source.RichSourceFunction;
+import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
+import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators;
+import org.apache.flink.streaming.connectors.kafka.testutils.DiscardingSink;
+import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
+import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils;
+import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
+import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper;
+import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
+import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper;
+import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2Partitioner;
+import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
+import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
+import org.apache.flink.testutils.junit.RetryOnException;
+import org.apache.flink.testutils.junit.RetryRule;
+import org.apache.flink.util.Collector;
+
+import org.apache.flink.util.NetUtils;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.junit.Assert;
+
+import org.junit.Rule;
+import scala.collection.Seq;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
+@SuppressWarnings("serial")
+public abstract class KafkaConsumerTestBase extends KafkaTestBase {
+	
+	@Rule
+	public RetryRule retryRule = new RetryRule();
+	
+	// ------------------------------------------------------------------------
+	//  Required methods by the abstract test base
+	// ------------------------------------------------------------------------
+
+	protected abstract <T> FlinkKafkaConsumer<T> getConsumer(
+			String topic, DeserializationSchema<T> deserializationSchema, Properties props);
+
+	// ------------------------------------------------------------------------
+	//  Suite of Tests
+	//
+	//  The tests here are all not activated (by an @Test tag), but need
+	//  to be invoked from the extending classes. That way, the classes can
+	//  select which tests to run.
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Test that validates that checkpointing and checkpoint notification works properly
+	 */
+	public void runCheckpointingTest() throws Exception {
+		createTestTopic("testCheckpointing", 1, 1);
+
+		FlinkKafkaConsumer<String> source = getConsumer("testCheckpointing", new JavaDefaultStringSchema(), standardProps);
+		Field pendingCheckpointsField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints");
+		pendingCheckpointsField.setAccessible(true);
+		LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source);
+
+		Assert.assertEquals(0, pendingCheckpoints.size());
+		source.setRuntimeContext(new MockRuntimeContext(1, 0));
+
+		final long[] initialOffsets = new long[] { 1337 };
+
+		// first restore
+		source.restoreState(initialOffsets);
+
+		// then open
+		source.open(new Configuration());
+		long[] state1 = source.snapshotState(1, 15);
+
+		assertArrayEquals(initialOffsets, state1);
+
+		long[] state2 = source.snapshotState(2, 30);
+		Assert.assertArrayEquals(initialOffsets, state2);
+		Assert.assertEquals(2, pendingCheckpoints.size());
+
+		source.notifyCheckpointComplete(1);
+		Assert.assertEquals(1, pendingCheckpoints.size());
+
+		source.notifyCheckpointComplete(2);
+		Assert.assertEquals(0, pendingCheckpoints.size());
+
+		source.notifyCheckpointComplete(666); // invalid checkpoint
+		Assert.assertEquals(0, pendingCheckpoints.size());
+
+		// create 500 snapshots
+		for (int i = 100; i < 600; i++) {
+			source.snapshotState(i, 15 * i);
+		}
+		Assert.assertEquals(FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size());
+
+		// commit only the second last
+		source.notifyCheckpointComplete(598);
+		Assert.assertEquals(1, pendingCheckpoints.size());
+
+		// access invalid checkpoint
+		source.notifyCheckpointComplete(590);
+
+		// and the last
+		source.notifyCheckpointComplete(599);
+		Assert.assertEquals(0, pendingCheckpoints.size());
+
+		source.close();
+
+		deleteTestTopic("testCheckpointing");
+	}
+
+	/**
+	 * Tests that offsets are properly committed to ZooKeeper and initial offsets are read from ZooKeeper.
+	 *
+	 * This test is only applicable if Teh Flink Kafka Consumer uses the ZooKeeperOffsetHandler.
+	 */
+	public void runOffsetInZookeeperValidationTest() throws Exception {
+		LOG.info("Starting testFlinkKafkaConsumerWithOffsetUpdates()");
+
+		final String topicName = "testOffsetHacking";
+		final int parallelism = 3;
+
+		createTestTopic(topicName, parallelism, 1);
+
+		StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env1.getConfig().disableSysoutLogging();
+		env1.enableCheckpointing(50);
+		env1.setNumberOfExecutionRetries(0);
+		env1.setParallelism(parallelism);
+
+		StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env2.getConfig().disableSysoutLogging();
+		env2.enableCheckpointing(50);
+		env2.setNumberOfExecutionRetries(0);
+		env2.setParallelism(parallelism);
+
+		StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env3.getConfig().disableSysoutLogging();
+		env3.enableCheckpointing(50);
+		env3.setNumberOfExecutionRetries(0);
+		env3.setParallelism(parallelism);
+
+		// write a sequence from 0 to 99 to each of the 3 partitions.
+		writeSequence(env1, topicName, 100, parallelism);
+
+		readSequence(env2, standardProps, parallelism, topicName, 100, 0);
+
+		ZkClient zkClient = createZookeeperClient();
+
+		long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 0);
+		long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 1);
+		long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 2);
+
+		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
+
+		assertTrue(o1 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+		assertTrue(o2 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+		assertTrue(o3 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+
+		LOG.info("Manipulating offsets");
+
+		// set the offset to 50 for the three partitions
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 0, 49);
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 1, 49);
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 2, 49);
+
+		zkClient.close();
+
+		// create new env
+		readSequence(env3, standardProps, parallelism, topicName, 50, 50);
+
+		deleteTestTopic(topicName);
+
+		LOG.info("Finished testFlinkKafkaConsumerWithOffsetUpdates()");
+	}
+
+	/**
+	 * Ensure Kafka is working on both producer and consumer side.
+	 * This executes a job that contains two Flink pipelines.
+	 *
+	 * <pre>
+	 * (generator source) --> (kafka sink)-[KAFKA-TOPIC]-(kafka source) --> (validating sink)
+	 * </pre>
+	 * 
+	 * We need to externally retry this test. We cannot let Flink's retry mechanism do it, because the Kafka producer
+	 * does not guarantee exactly-once output. Hence a recovery would introduce duplicates that
+	 * cause the test to fail.
+	 */
+	@RetryOnException(times=2, exception=kafka.common.NotLeaderForPartitionException.class)
+	public void runSimpleConcurrentProducerConsumerTopology() throws Exception {
+		LOG.info("Starting runSimpleConcurrentProducerConsumerTopology()");
+
+		final String topic = "concurrentProducerConsumerTopic_" + UUID.randomUUID().toString();
+		final int parallelism = 3;
+		final int elementsPerPartition = 100;
+		final int totalElements = parallelism * elementsPerPartition;
+
+		createTestTopic(topic, parallelism, 2);
+
+		final StreamExecutionEnvironment env =
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(parallelism);
+		env.getConfig().disableSysoutLogging();
+
+		TypeInformation<Tuple2<Long, String>> longStringType = TypeInfoParser.parse("Tuple2<Long, String>");
+
+		TypeInformationSerializationSchema<Tuple2<Long, String>> sourceSchema =
+				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
+
+		TypeInformationSerializationSchema<Tuple2<Long, String>> sinkSchema =
+				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
+
+		// ----------- add producer dataflow ----------
+
+		DataStream<Tuple2<Long, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Long,String>>() {
+
+			private boolean running = true;
+
+			@Override
+			public void run(SourceContext<Tuple2<Long, String>> ctx) {
+				int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
+				int limit = cnt + elementsPerPartition;
+
+
+				while (running && cnt < limit) {
+					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt));
+					cnt++;
+				}
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+		stream.addSink(new FlinkKafkaProducer<>(brokerConnectionStrings, topic, sinkSchema));
+
+		// ----------- add consumer dataflow ----------
+
+		FlinkKafkaConsumer<Tuple2<Long, String>> source = getConsumer(topic, sourceSchema, standardProps);
+
+		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source).setParallelism(parallelism);
+
+		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
+
+			private int elCnt = 0;
+			private BitSet validator = new BitSet(totalElements);
+
+			@Override
+			public void invoke(Tuple2<Long, String> value) throws Exception {
+				String[] sp = value.f1.split("-");
+				int v = Integer.parseInt(sp[1]);
+
+				assertEquals(value.f0 - 1000, (long) v);
+
+				assertFalse("Received tuple twice", validator.get(v));
+				validator.set(v);
+				elCnt++;
+
+				if (elCnt == totalElements) {
+					// check if everything in the bitset is set to true
+					int nc;
+					if ((nc = validator.nextClearBit(0)) != totalElements) {
+						fail("The bitset was not set to 1 on all elements. Next clear:"
+								+ nc + " Set: " + validator);
+					}
+					throw new SuccessException();
+				}
+			}
+
+			@Override
+			public void close() throws Exception {
+				super.close();
+			}
+		}).setParallelism(1);
+
+		try {
+			tryExecutePropagateExceptions(env, "runSimpleConcurrentProducerConsumerTopology");
+		}
+		catch (ProgramInvocationException | JobExecutionException e) {
+			// look for NotLeaderForPartitionException
+			Throwable cause = e.getCause();
+
+			// search for nested SuccessExceptions
+			int depth = 0;
+			while (cause != null && depth++ < 20) {
+				if (cause instanceof kafka.common.NotLeaderForPartitionException) {
+					throw (Exception) cause;
+				}
+				cause = cause.getCause();
+			}
+			throw e;
+		}
+
+		LOG.info("Finished runSimpleConcurrentProducerConsumerTopology()");
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and
+	 * Flink sources.
+	 */
+	public void runOneToOneExactlyOnceTest() throws Exception {
+		LOG.info("Starting runOneToOneExactlyOnceTest()");
+
+		final String topic = "oneToOneTopic";
+		final int parallelism = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = parallelism * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+		createTestTopic(topic, parallelism, 1);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, parallelism, numElementsPerPartition, true);
+
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(parallelism, 1))
+				.map(new FailingIdentityMapper<Integer>(failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "One-to-one exactly once test");
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests the proper consumption when having fewer Flink sources than Kafka partitions, so
+	 * one Flink source will read multiple Kafka partitions.
+	 */
+	public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception {
+		LOG.info("Starting runOneSourceMultiplePartitionsExactlyOnceTest()");
+
+		final String topic = "oneToManyTopic";
+		final int numPartitions = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = numPartitions * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+		final int parallelism = 2;
+
+		createTestTopic(topic, numPartitions, 1);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, numPartitions, numElementsPerPartition, true);
+
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(numPartitions, 3))
+				.map(new FailingIdentityMapper<Integer>(failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "One-source-multi-partitions exactly once test");
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests the proper consumption when having more Flink sources than Kafka partitions, which means
+	 * that some Flink sources will read no partitions.
+	 */
+	public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception {
+		LOG.info("Starting runMultipleSourcesOnePartitionExactlyOnceTest()");
+
+		final String topic = "manyToOneTopic";
+		final int numPartitions = 5;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = numPartitions * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+		final int parallelism = 8;
+
+		createTestTopic(topic, numPartitions, 1);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, numPartitions, numElementsPerPartition, true);
+
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.enableCheckpointing(500);
+		env.setParallelism(parallelism);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
+		env.setBufferTimeout(0);
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+			.addSource(kafkaSource)
+			.map(new PartitionValidatingMapper(numPartitions, 1))
+			.map(new FailingIdentityMapper<Integer>(failAfterElements))
+			.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		FailingIdentityMapper.failedBefore = false;
+		tryExecute(env, "multi-source-one-partitions exactly once test");
+
+
+		deleteTestTopic(topic);
+	}
+	
+	
+	/**
+	 * Tests that the source can be properly canceled when reading full partitions. 
+	 */
+	public void runCancelingOnFullInputTest() throws Exception {
+		final String topic = "cancelingOnFullTopic";
+
+		final int parallelism = 3;
+		createTestTopic(topic, parallelism, 1);
+
+		// launch a producer thread
+		DataGenerators.InfiniteStringsGenerator generator =
+				new DataGenerators.InfiniteStringsGenerator(brokerConnectionStrings, topic);
+		generator.start();
+
+		// launch a consumer asynchronously
+
+		final AtomicReference<Throwable> jobError = new AtomicReference<>();
+
+		final Runnable jobRunner = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					env.setParallelism(parallelism);
+					env.enableCheckpointing(100);
+					env.getConfig().disableSysoutLogging();
+
+					FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
+
+					env.addSource(source).addSink(new DiscardingSink<String>());
+
+					env.execute();
+				}
+				catch (Throwable t) {
+					jobError.set(t);
+				}
+			}
+		};
+
+		Thread runnerThread = new Thread(jobRunner, "program runner thread");
+		runnerThread.start();
+
+		// wait a bit before canceling
+		Thread.sleep(2000);
+
+		// cancel
+		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
+
+		// wait for the program to be done and validate that we failed with the right exception
+		runnerThread.join();
+
+		Throwable failueCause = jobError.get();
+		assertNotNull("program did not fail properly due to canceling", failueCause);
+		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
+
+		if (generator.isAlive()) {
+			generator.shutdown();
+			generator.join();
+		}
+		else {
+			Throwable t = generator.getError();
+			if (t != null) {
+				t.printStackTrace();
+				fail("Generator failed: " + t.getMessage());
+			} else {
+				fail("Generator failed with no exception");
+			}
+		}
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests that the source can be properly canceled when reading empty partitions. 
+	 */
+	public void runCancelingOnEmptyInputTest() throws Exception {
+		final String topic = "cancelingOnEmptyInputTopic";
+
+		final int parallelism = 3;
+		createTestTopic(topic, parallelism, 1);
+
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final Runnable jobRunner = new Runnable() {
+			@Override
+			public void run() {
+				try {
+					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+					env.setParallelism(parallelism);
+					env.enableCheckpointing(100);
+					env.getConfig().disableSysoutLogging();
+
+					FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
+
+					env.addSource(source).addSink(new DiscardingSink<String>());
+
+					env.execute();
+				}
+				catch (Throwable t) {
+					error.set(t);
+				}
+			}
+		};
+
+		Thread runnerThread = new Thread(jobRunner, "program runner thread");
+		runnerThread.start();
+
+		// wait a bit before canceling
+		Thread.sleep(2000);
+
+		// cancel
+		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
+
+		// wait for the program to be done and validate that we failed with the right exception
+		runnerThread.join();
+
+		Throwable failueCause = error.get();
+		assertNotNull("program did not fail properly due to canceling", failueCause);
+		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Tests that the source can be properly canceled when reading full partitions. 
+	 */
+	public void runFailOnDeployTest() throws Exception {
+		final String topic = "failOnDeployTopic";
+
+		createTestTopic(topic, 2, 1);
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(12); // needs to be more that the mini cluster has slots
+		env.getConfig().disableSysoutLogging();
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+				.addSource(kafkaSource)
+				.addSink(new DiscardingSink<Integer>());
+
+		try {
+			env.execute();
+			fail("this test should fail with an exception");
+		}
+		catch (ProgramInvocationException e) {
+
+			// validate that we failed due to a NoResourceAvailableException
+			Throwable cause = e.getCause();
+			int depth = 0;
+			boolean foundResourceException = false;
+
+			while (cause != null && depth++ < 20) {
+				if (cause instanceof NoResourceAvailableException) {
+					foundResourceException = true;
+					break;
+				}
+				cause = cause.getCause();
+			}
+
+			assertTrue("Wrong exception", foundResourceException);
+		}
+
+		deleteTestTopic(topic);
+	}
+
+	public void runInvalidOffsetTest() throws Exception {
+		final String topic = "invalidOffsetTopic";
+		final int parallelism = 1;
+
+		// create topic
+		createTestTopic(topic, parallelism, 1);
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+
+		// write 20 messages into topic:
+		writeSequence(env, topic, 20, parallelism);
+
+		// set invalid offset:
+		ZkClient zkClient = createZookeeperClient();
+		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topic, 0, 1234);
+
+		// read from topic
+		final int valuesCount = 20;
+		final int startFrom = 0;
+		readSequence(env, standardCC.props().props(), parallelism, topic, valuesCount, startFrom);
+
+		deleteTestTopic(topic);
+	}
+
+	/**
+	 * Test Flink's Kafka integration also with very big records (30MB)
+	 * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
+	 */
+	public void runBigRecordTestTopology() throws Exception {
+		LOG.info("Starting runBigRecordTestTopology()");
+
+		final String topic = "bigRecordTestTopic";
+		final int parallelism = 1; // otherwise, the kafka mini clusters may run out of heap space
+
+		createTestTopic(topic, parallelism, 1);
+
+		final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
+
+		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
+				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
+
+		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> deserSchema =
+				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setNumberOfExecutionRetries(0);
+		env.getConfig().disableSysoutLogging();
+		env.enableCheckpointing(100);
+		env.setParallelism(parallelism);
+
+		// add consuming topology:
+		Properties consumerProps = new Properties();
+		consumerProps.putAll(standardProps);
+		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 40));
+		consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 40)); // for the new fetcher
+		consumerProps.setProperty("queued.max.message.chunks", "1");
+
+		FlinkKafkaConsumer<Tuple2<Long, byte[]>> source = getConsumer(topic, serSchema, consumerProps);
+		DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(source);
+
+		consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
+
+			private int elCnt = 0;
+
+			@Override
+			public void invoke(Tuple2<Long, byte[]> value) throws Exception {
+				elCnt++;
+				if (value.f0 == -1) {
+					// we should have seen 11 elements now.
+					if(elCnt == 11) {
+						throw new SuccessException();
+					} else {
+						throw new RuntimeException("There have been "+elCnt+" elements");
+					}
+				}
+				if(elCnt > 10) {
+					throw new RuntimeException("More than 10 elements seen: "+elCnt);
+				}
+			}
+		});
+
+		// add producing topology
+		Properties producerProps = new Properties();
+		producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 30));
+		producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerConnectionStrings);
+
+		DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
+
+			private boolean running;
+
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				super.open(parameters);
+				running = true;
+			}
+
+			@Override
+			public void run(SourceContext<Tuple2<Long, byte[]>> ctx) throws Exception {
+				Random rnd = new Random();
+				long cnt = 0;
+				int fifteenMb = 1024 * 1024 * 15;
+
+				while (running) {
+					byte[] wl = new byte[fifteenMb + rnd.nextInt(fifteenMb)];
+					ctx.collect(new Tuple2<Long, byte[]>(cnt++, wl));
+
+					Thread.sleep(100);
+
+					if (cnt == 10) {
+						// signal end
+						ctx.collect(new Tuple2<Long, byte[]>(-1L, new byte[]{1}));
+						break;
+					}
+				}
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+
+		stream.addSink(new FlinkKafkaProducer<>(topic, deserSchema, producerProps));
+
+		tryExecute(env, "big topology test");
+
+		deleteTestTopic(topic);
+
+		LOG.info("Finished runBigRecordTestTopology()");
+	}
+
+	
+	public void runBrokerFailureTest() throws Exception {
+		LOG.info("starting runBrokerFailureTest()");
+
+		final String topic = "brokerFailureTestTopic";
+
+		final int parallelism = 2;
+		final int numElementsPerPartition = 1000;
+		final int totalElements = parallelism * numElementsPerPartition;
+		final int failAfterElements = numElementsPerPartition / 3;
+
+
+		createTestTopic(topic, parallelism, 2);
+
+		DataGenerators.generateRandomizedIntegerSequence(
+				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
+				brokerConnectionStrings,
+				topic, parallelism, numElementsPerPartition, true);
+
+		// find leader to shut down
+		ZkClient zkClient = createZookeeperClient();
+		PartitionMetadata firstPart = null;
+		do {
+			if (firstPart != null) {
+				LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
+				// not the first try. Sleep a bit
+				Thread.sleep(150);
+			}
+
+			Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata();
+			firstPart = partitionMetadata.head();
+		}
+		while (firstPart.errorCode() != 0);
+		zkClient.close();
+
+		final kafka.cluster.Broker leaderToShutDown = firstPart.leader().get();
+		final String leaderToShutDownConnection = 
+				NetUtils.hostAndPortToUrlString(leaderToShutDown.host(), leaderToShutDown.port());
+		
+		
+		final int leaderIdToShutDown = firstPart.leader().get().id();
+		LOG.info("Leader to shutdown {}", leaderToShutDown);
+
+
+		// run the topology that fails and recovers
+
+		DeserializationSchema<Integer> schema =
+				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setParallelism(parallelism);
+		env.enableCheckpointing(500);
+		env.setNumberOfExecutionRetries(3);
+		env.getConfig().disableSysoutLogging();
+
+
+		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
+
+		env
+				.addSource(kafkaSource)
+				.map(new PartitionValidatingMapper(parallelism, 1))
+				.map(new BrokerKillingMapper<Integer>(leaderToShutDownConnection, failAfterElements))
+				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
+
+		BrokerKillingMapper.killedLeaderBefore = false;
+		tryExecute(env, "One-to-one exactly once test");
+
+		// start a new broker:
+		brokers.set(leaderIdToShutDown, getKafkaServer(leaderIdToShutDown, tmpKafkaDirs.get(leaderIdToShutDown), kafkaHost, zookeeperConnectionString));
+
+		LOG.info("finished runBrokerFailureTest()");
+	}
+
+	// ------------------------------------------------------------------------
+	//  Reading writing test data sets
+	// ------------------------------------------------------------------------
+
+	private void readSequence(StreamExecutionEnvironment env, Properties cc,
+								final int sourceParallelism,
+								final String topicName,
+								final int valuesCount, final int startFrom) throws Exception {
+
+		final int finalCount = valuesCount * sourceParallelism;
+
+		final TypeInformation<Tuple2<Integer, Integer>> intIntTupleType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+
+		final TypeInformationSerializationSchema<Tuple2<Integer, Integer>> deser =
+				new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig());
+
+		// create the consumer
+		FlinkKafkaConsumer<Tuple2<Integer, Integer>> consumer = getConsumer(topicName, deser, cc);
+
+		DataStream<Tuple2<Integer, Integer>> source = env
+				.addSource(consumer).setParallelism(sourceParallelism)
+				.map(new ThrottledMapper<Tuple2<Integer, Integer>>(20)).setParallelism(sourceParallelism);
+
+		// verify data
+		source.flatMap(new RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>() {
+
+			private int[] values = new int[valuesCount];
+			private int count = 0;
+
+			@Override
+			public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
+				values[value.f1 - startFrom]++;
+				count++;
+
+				// verify if we've seen everything
+				if (count == finalCount) {
+					for (int i = 0; i < values.length; i++) {
+						int v = values[i];
+						if (v != sourceParallelism) {
+							printTopic(topicName, valuesCount, deser);
+							throw new RuntimeException("Expected v to be 3, but was " + v + " on element " + i + " array=" + Arrays.toString(values));
+						}
+					}
+					// test has passed
+					throw new SuccessException();
+				}
+			}
+
+		}).setParallelism(1);
+
+		tryExecute(env, "Read data from Kafka");
+
+		LOG.info("Successfully read sequence for verification");
+	}
+
+	private static void writeSequence(StreamExecutionEnvironment env, String topicName, final int numElements, int parallelism) throws Exception {
+
+		TypeInformation<Tuple2<Integer, Integer>> resultType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+
+		DataStream<Tuple2<Integer, Integer>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
+
+			private boolean running = true;
+
+			@Override
+			public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
+				int cnt = 0;
+				int partition = getRuntimeContext().getIndexOfThisSubtask();
+
+				while (running && cnt < numElements) {
+					ctx.collect(new Tuple2<Integer, Integer>(partition, cnt));
+					cnt++;
+				}
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		}).setParallelism(parallelism);
+		
+		stream.addSink(new FlinkKafkaProducer<>(topicName,
+				new TypeInformationSerializationSchema<>(resultType, env.getConfig()),
+				FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings),
+				new Tuple2Partitioner(parallelism)
+		)).setParallelism(parallelism);
+
+		env.execute("Write sequence");
+
+		LOG.info("Finished writing sequence");
+	}
+
+	// ------------------------------------------------------------------------
+	//  Debugging utilities
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Read topic to list, only using Kafka code.
+	 */
+	private static List<MessageAndMetadata<byte[], byte[]>> readTopicToList(String topicName, ConsumerConfig config, final int stopAfter) {
+		ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(config);
+		// we request only one stream per consumer instance. Kafka will make sure that each consumer group
+		// will see each message only once.
+		Map<String,Integer> topicCountMap = Collections.singletonMap(topicName, 1);
+		Map<String, List<KafkaStream<byte[], byte[]>>> streams = consumerConnector.createMessageStreams(topicCountMap);
+		if(streams.size() != 1) {
+			throw new RuntimeException("Expected only one message stream but got "+streams.size());
+		}
+		List<KafkaStream<byte[], byte[]>> kafkaStreams = streams.get(topicName);
+		if(kafkaStreams == null) {
+			throw new RuntimeException("Requested stream not available. Available streams: "+streams.toString());
+		}
+		if(kafkaStreams.size() != 1) {
+			throw new RuntimeException("Requested 1 stream from Kafka, bot got "+kafkaStreams.size()+" streams");
+		}
+		LOG.info("Opening Consumer instance for topic '{}' on group '{}'", topicName, config.groupId());
+		ConsumerIterator<byte[], byte[]> iteratorToRead = kafkaStreams.get(0).iterator();
+
+		List<MessageAndMetadata<byte[], byte[]>> result = new ArrayList<MessageAndMetadata<byte[], byte[]>>();
+		int read = 0;
+		while(iteratorToRead.hasNext()) {
+			read++;
+			result.add(iteratorToRead.next());
+			if(read == stopAfter) {
+				LOG.info("Read "+read+" elements");
+				return result;
+			}
+		}
+		return result;
+	}
+
+	private static void printTopic(String topicName, ConsumerConfig config,
+								DeserializationSchema<?> deserializationSchema,
+								int stopAfter) {
+
+		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
+		LOG.info("Printing contents of topic {} in consumer grouo {}", topicName, config.groupId());
+
+		for (MessageAndMetadata<byte[], byte[]> message: contents) {
+			Object out = deserializationSchema.deserialize(message.message());
+			LOG.info("Message: partition: {} offset: {} msg: {}", message.partition(), message.offset(), out.toString());
+		}
+	}
+
+	private static void printTopic(String topicName, int elements,DeserializationSchema<?> deserializer) {
+		// write the sequence to log for debugging purposes
+		Properties stdProps = standardCC.props().props();
+		Properties newProps = new Properties(stdProps);
+		newProps.setProperty("group.id", "topic-printer"+ UUID.randomUUID().toString());
+		newProps.setProperty("auto.offset.reset", "smallest");
+		newProps.setProperty("zookeeper.connect", standardCC.zkConnect());
+
+		ConsumerConfig printerConfig = new ConsumerConfig(newProps);
+		printTopic(topicName, printerConfig, deserializer, elements);
+	}
+
+
+	public static class BrokerKillingMapper<T> extends RichMapFunction<T,T>
+			implements Checkpointed<Integer>, CheckpointNotifier {
+
+		private static final long serialVersionUID = 6334389850158707313L;
+
+		public static volatile boolean killedLeaderBefore;
+		public static volatile boolean hasBeenCheckpointedBeforeFailure;
+		
+		private final String leaderToShutDown;
+		private final int failCount;
+		private int numElementsTotal;
+
+		private boolean failer;
+		private boolean hasBeenCheckpointed;
+
+
+		public BrokerKillingMapper(String leaderToShutDown, int failCount) {
+			this.leaderToShutDown = leaderToShutDown;
+			this.failCount = failCount;
+		}
+
+		@Override
+		public void open(Configuration parameters) {
+			failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
+		}
+
+		@Override
+		public T map(T value) throws Exception {
+			numElementsTotal++;
+			
+			if (!killedLeaderBefore) {
+				Thread.sleep(10);
+				
+				if (failer && numElementsTotal >= failCount) {
+					// shut down a Kafka broker
+					KafkaServer toShutDown = null;
+					for (KafkaServer kafkaServer : brokers) {
+						String connectionUrl = 
+								NetUtils.hostAndPortToUrlString(
+										kafkaServer.config().advertisedHostName(),
+										kafkaServer.config().advertisedPort());
+						if (leaderToShutDown.equals(connectionUrl)) {
+							toShutDown = kafkaServer;
+							break;
+						}
+					}
+	
+					if (toShutDown == null) {
+						StringBuilder listOfBrokers = new StringBuilder();
+						for (KafkaServer kafkaServer : brokers) {
+							listOfBrokers.append(
+									NetUtils.hostAndPortToUrlString(
+											kafkaServer.config().advertisedHostName(),
+											kafkaServer.config().advertisedPort()));
+							listOfBrokers.append(" ; ");
+						}
+						
+						throw new Exception("Cannot find broker to shut down: " + leaderToShutDown
+								+ " ; available brokers: " + listOfBrokers.toString());
+					}
+					else {
+						hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed;
+						killedLeaderBefore = true;
+						toShutDown.shutdown();
+					}
+				}
+			}
+			return value;
+		}
+
+		@Override
+		public void notifyCheckpointComplete(long checkpointId) {
+			hasBeenCheckpointed = true;
+		}
+
+		@Override
+		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+			return numElementsTotal;
+		}
+
+		@Override
+		public void restoreState(Integer state) {
+			this.numElementsTotal = state;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
new file mode 100644
index 0000000..b4511ce
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import org.junit.Test;
+
+import java.util.Properties;
+
+
+public class KafkaITCase extends KafkaConsumerTestBase {
+	
+	@Override
+	protected <T> FlinkKafkaConsumer<T> getConsumer(String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
+		return new FlinkKafkaConsumer081<>(topic, deserializationSchema, props);
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Suite of Tests
+	// ------------------------------------------------------------------------
+	
+	@Test
+	public void testCheckpointing() throws Exception {
+		runCheckpointingTest();
+	}
+
+	@Test
+	public void testOffsetInZookeeper() throws Exception {
+		runOffsetInZookeeperValidationTest();
+	}
+	
+	@Test
+	public void testConcurrentProducerConsumerTopology() throws Exception {
+		runSimpleConcurrentProducerConsumerTopology();
+	}
+
+	// --- canceling / failures ---
+	
+	@Test
+	public void testCancelingEmptyTopic() throws Exception {
+		runCancelingOnEmptyInputTest();
+	}
+
+	@Test
+	public void testCancelingFullTopic() throws Exception {
+		runCancelingOnFullInputTest();
+	}
+
+	@Test
+	public void testFailOnDeploy() throws Exception {
+		runFailOnDeployTest();
+	}
+
+	@Test
+	public void testInvalidOffset() throws Exception {
+		runInvalidOffsetTest();
+	}
+
+	// --- source to partition mappings and exactly once ---
+	
+	@Test
+	public void testOneToOneSources() throws Exception {
+		runOneToOneExactlyOnceTest();
+	}
+
+	@Test
+	public void testOneSourceMultiplePartitions() throws Exception {
+		runOneSourceMultiplePartitionsExactlyOnceTest();
+	}
+
+	@Test
+	public void testMultipleSourcesOnePartition() throws Exception {
+		runMultipleSourcesOnePartitionExactlyOnceTest();
+	}
+
+	// --- broker failure ---
+
+	@Test
+	public void testBrokerFailure() throws Exception {
+		runBrokerFailureTest();
+	}
+
+	// --- special executions ---
+	
+	@Test
+	public void testBigRecordJob() throws Exception {
+		runBigRecordTestTopology();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
new file mode 100644
index 0000000..72d2772
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+import kafka.utils.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KafkaLocalSystemTime implements Time {
+
+	private static final Logger LOG = LoggerFactory.getLogger(KafkaLocalSystemTime.class);
+
+	@Override
+	public long milliseconds() {
+		return System.currentTimeMillis();
+	}
+
+	@Override
+	public long nanoseconds() {
+		return System.nanoTime();
+	}
+
+	@Override
+	public void sleep(long ms) {
+		try {
+			Thread.sleep(ms);
+		} catch (InterruptedException e) {
+			LOG.warn("Interruption", e);
+		}
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java
new file mode 100644
index 0000000..5001364
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.streaming.api.datastream.DataStream;
+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.SourceFunction;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
+import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
+
+import org.junit.Test;
+
+import java.io.Serializable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+@SuppressWarnings("serial")
+public class KafkaProducerITCase extends KafkaTestBase {
+
+
+	/**
+	 * 
+	 * <pre>
+	 *             +------> (sink) --+--> [KAFKA-1] --> (source) -> (map) --+
+	 *            /                  |                                       \
+	 *           /                   |                                        \
+	 * (source) ----------> (sink) --+--> [KAFKA-2] --> (source) -> (map) -----+-> (sink)
+	 *           \                   |                                        /
+	 *            \                  |                                       /
+	 *             +------> (sink) --+--> [KAFKA-3] --> (source) -> (map) --+
+	 * </pre>
+	 * 
+	 * The mapper validates that the values come consistently from the correct Kafka partition.
+	 * 
+	 * The final sink validates that there are no duplicates and that all partitions are present.
+	 */
+	@Test
+	public void testCustomPartitioning() {
+		try {
+			LOG.info("Starting KafkaProducerITCase.testCustomPartitioning()");
+
+			final String topic = "customPartitioningTestTopic";
+			final int parallelism = 3;
+			
+			createTestTopic(topic, parallelism, 1);
+
+			TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
+
+			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			env.setNumberOfExecutionRetries(0);
+			env.getConfig().disableSysoutLogging();
+
+			TypeInformationSerializationSchema<Tuple2<Long, String>> serSchema =
+					new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig());
+
+			TypeInformationSerializationSchema<Tuple2<Long, String>> deserSchema =
+					new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig());
+
+			// ------ producing topology ---------
+			
+			// source has DOP 1 to make sure it generates no duplicates
+			DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
+
+				private boolean running = true;
+
+				@Override
+				public void run(SourceContext<Tuple2<Long, String>> ctx) throws Exception {
+					long cnt = 0;
+					while (running) {
+						ctx.collect(new Tuple2<Long, String>(cnt, "kafka-" + cnt));
+						cnt++;
+					}
+				}
+
+				@Override
+				public void cancel() {
+					running = false;
+				}
+			})
+			.setParallelism(1);
+			
+			// sink partitions into 
+			stream.addSink(new FlinkKafkaProducer<>(topic, serSchema, FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings), new CustomPartitioner(parallelism)))
+			.setParallelism(parallelism);
+
+			// ------ consuming topology ---------
+			
+			FlinkKafkaConsumer<Tuple2<Long, String>> source = 
+					new FlinkKafkaConsumer<>(topic, deserSchema, standardProps, 
+							FlinkKafkaConsumer.OffsetStore.FLINK_ZOOKEEPER,
+							FlinkKafkaConsumer.FetcherType.LEGACY_LOW_LEVEL);
+			
+			env.addSource(source).setParallelism(parallelism)
+
+					// mapper that validates partitioning and maps to partition
+					.map(new RichMapFunction<Tuple2<Long, String>, Integer>() {
+						
+						private int ourPartition = -1;
+						@Override
+						public Integer map(Tuple2<Long, String> value) {
+							int partition = value.f0.intValue() % parallelism;
+							if (ourPartition != -1) {
+								assertEquals("inconsistent partitioning", ourPartition, partition);
+							} else {
+								ourPartition = partition;
+							}
+							return partition;
+						}
+					}).setParallelism(parallelism)
+					
+					.addSink(new SinkFunction<Integer>() {
+						
+						private int[] valuesPerPartition = new int[parallelism];
+						
+						@Override
+						public void invoke(Integer value) throws Exception {
+							valuesPerPartition[value]++;
+							
+							boolean missing = false;
+							for (int i : valuesPerPartition) {
+								if (i < 100) {
+									missing = true;
+									break;
+								}
+							}
+							if (!missing) {
+								throw new SuccessException();
+							}
+						}
+					}).setParallelism(1);
+			
+			tryExecute(env, "custom partitioning test");
+
+			deleteTestTopic(topic);
+			
+			LOG.info("Finished KafkaProducerITCase.testCustomPartitioning()");
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	
+	// ------------------------------------------------------------------------
+
+	public static class CustomPartitioner extends KafkaPartitioner implements Serializable {
+
+		private final int expectedPartitions;
+
+		public CustomPartitioner(int expectedPartitions) {
+			this.expectedPartitions = expectedPartitions;
+		}
+
+		@Override
+		public int partition(Object key, int numPartitions) {
+			@SuppressWarnings("unchecked")
+			Tuple2<Long, String> tuple = (Tuple2<Long, String>) key;
+			
+			assertEquals(expectedPartitions, numPartitions);
+			
+			return (int) (tuple.f0 % numPartitions);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
new file mode 100644
index 0000000..c5c3387
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
+import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.PartitionInfo;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.Arrays;
+import java.util.Properties;
+import java.util.concurrent.Future;
+
+
+import static org.mockito.Mockito.*;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import static org.junit.Assert.*;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(FlinkKafkaProducer.class)
+public class KafkaProducerTest extends TestLogger {
+	
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testPropagateExceptions() {
+		try {
+			// mock kafka producer
+			KafkaProducer<?, ?> kafkaProducerMock = mock(KafkaProducer.class);
+			
+			// partition setup
+			when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
+					Arrays.asList(new PartitionInfo("mock_topic", 42, null, null, null)));
+
+			// failure when trying to send an element
+			when(kafkaProducerMock.send(any(ProducerRecord.class), any(Callback.class)))
+				.thenAnswer(new Answer<Future<RecordMetadata>>() {
+					@Override
+					public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
+						Callback callback = (Callback) invocation.getArguments()[1];
+						callback.onCompletion(null, new Exception("Test error"));
+						return null;
+					}
+				});
+			
+			// make sure the FlinkKafkaProducer instantiates our mock producer
+			whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
+			
+			// (1) producer that propagates errors
+			
+			FlinkKafkaProducer<String> producerPropagating = new FlinkKafkaProducer<String>(
+					"mock_topic", new JavaDefaultStringSchema(), new Properties(), null);
+
+			producerPropagating.setRuntimeContext(new MockRuntimeContext(17, 3));
+			producerPropagating.open(new Configuration());
+			
+			try {
+				producerPropagating.invoke("value");
+				producerPropagating.invoke("value");
+				fail("This should fail with an exception");
+			}
+			catch (Exception e) {
+				assertNotNull(e.getCause());
+				assertNotNull(e.getCause().getMessage());
+				assertTrue(e.getCause().getMessage().contains("Test error"));
+			}
+
+			// (2) producer that only logs errors
+			
+			FlinkKafkaProducer<String> producerLogging = new FlinkKafkaProducer<String>(
+					"mock_topic", new JavaDefaultStringSchema(), new Properties(), null);
+			producerLogging.setLogFailuresOnly(true);
+			
+			producerLogging.setRuntimeContext(new MockRuntimeContext(17, 3));
+			producerLogging.open(new Configuration());
+
+			producerLogging.invoke("value");
+			producerLogging.invoke("value");
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
new file mode 100644
index 0000000..d511796
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
@@ -0,0 +1,382 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+import kafka.admin.AdminUtils;
+import kafka.common.KafkaException;
+import kafka.consumer.ConsumerConfig;
+import kafka.network.SocketServer;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServer;
+
+import org.I0Itec.zkclient.ZkClient;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSerializer;
+import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
+import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.util.NetUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.kafka.common.PartitionInfo;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.BindException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * The base for the Kafka tests. It brings up:
+ * <ul>
+ *     <li>A ZooKeeper mini cluster</li>
+ *     <li>Three Kafka Brokers (mini clusters)</li>
+ *     <li>A Flink mini cluster</li>
+ * </ul>
+ * 
+ * <p>Code in this test is based on the following GitHub repository:
+ * <a href="https://github.com/sakserv/hadoop-mini-clusters">
+ *   https://github.com/sakserv/hadoop-mini-clusters</a> (ASL licensed),
+ * as per commit <i>bc6b2b2d5f6424d5f377aa6c0871e82a956462ef</i></p>
+ */
+@SuppressWarnings("serial")
+public abstract class KafkaTestBase extends TestLogger {
+
+	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
+	
+	protected static final int NUMBER_OF_KAFKA_SERVERS = 3;
+
+	protected static String zookeeperConnectionString;
+
+	protected static File tmpZkDir;
+
+	protected static File tmpKafkaParent;
+
+	protected static TestingServer zookeeper;
+	protected static List<KafkaServer> brokers;
+	protected static String brokerConnectionStrings = "";
+
+	protected static ConsumerConfig standardCC;
+	protected static Properties standardProps;
+	
+	protected static ForkableFlinkMiniCluster flink;
+
+	protected static int flinkPort;
+
+	protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
+
+	protected static List<File> tmpKafkaDirs;
+
+	protected static String kafkaHost = "localhost";
+
+	// ------------------------------------------------------------------------
+	//  Setup and teardown of the mini clusters
+	// ------------------------------------------------------------------------
+	
+	@BeforeClass
+	public static void prepare() throws IOException {
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Starting KafkaITCase ");
+		LOG.info("-------------------------------------------------------------------------");
+		
+		LOG.info("Starting KafkaITCase.prepare()");
+		
+		File tempDir = new File(System.getProperty("java.io.tmpdir"));
+		
+		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
+		assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
+
+		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
+		assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
+
+		tmpKafkaDirs = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS);
+		for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
+			File tmpDir = new File(tmpKafkaParent, "server-" + i);
+			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
+			tmpKafkaDirs.add(tmpDir);
+		}
+		
+		zookeeper = null;
+		brokers = null;
+
+		try {
+			LOG.info("Starting Zookeeper");
+			zookeeper = new TestingServer(-1, tmpZkDir);
+			zookeeperConnectionString = zookeeper.getConnectString();
+
+			LOG.info("Starting KafkaServer");
+			brokers = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS);
+			
+			for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
+				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), kafkaHost, zookeeperConnectionString));
+				SocketServer socketServer = brokers.get(i).socketServer();
+				
+				String host = socketServer.host() == null ? "localhost" : socketServer.host();
+				brokerConnectionStrings += hostAndPortToUrlString(host, socketServer.port()) + ",";
+			}
+
+			LOG.info("ZK and KafkaServer started.");
+		}
+		catch (Throwable t) {
+			t.printStackTrace();
+			fail("Test setup failed: " + t.getMessage());
+		}
+
+		standardProps = new Properties();
+
+		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
+		standardProps.setProperty("bootstrap.servers", brokerConnectionStrings);
+		standardProps.setProperty("group.id", "flink-tests");
+		standardProps.setProperty("auto.commit.enable", "false");
+		standardProps.setProperty("zookeeper.session.timeout.ms", "12000"); // 6 seconds is default. Seems to be too small for travis.
+		standardProps.setProperty("zookeeper.connection.timeout.ms", "20000");
+		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
+		standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
+		
+		Properties consumerConfigProps = new Properties();
+		consumerConfigProps.putAll(standardProps);
+		consumerConfigProps.setProperty("auto.offset.reset", "smallest");
+		standardCC = new ConsumerConfig(consumerConfigProps);
+		
+		// start also a re-usable Flink mini cluster
+		
+		Configuration flinkConfig = new Configuration();
+		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
+		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
+		flinkConfig.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 s");
+
+		flink = new ForkableFlinkMiniCluster(flinkConfig, false, StreamingMode.STREAMING);
+		flink.start();
+
+		flinkPort = flink.getLeaderRPCPort();
+	}
+
+	@AfterClass
+	public static void shutDownServices() {
+
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    Shut down KafkaITCase ");
+		LOG.info("-------------------------------------------------------------------------");
+
+		flinkPort = -1;
+		if (flink != null) {
+			flink.shutdown();
+		}
+		
+		for (KafkaServer broker : brokers) {
+			if (broker != null) {
+				broker.shutdown();
+			}
+		}
+		brokers.clear();
+		
+		if (zookeeper != null) {
+			try {
+				zookeeper.stop();
+			}
+			catch (Exception e) {
+				LOG.warn("ZK.stop() failed", e);
+			}
+			zookeeper = null;
+		}
+		
+		// clean up the temp spaces
+		
+		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpKafkaParent);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+		if (tmpZkDir != null && tmpZkDir.exists()) {
+			try {
+				FileUtils.deleteDirectory(tmpZkDir);
+			}
+			catch (Exception e) {
+				// ignore
+			}
+		}
+
+		LOG.info("-------------------------------------------------------------------------");
+		LOG.info("    KafkaITCase finished"); 
+		LOG.info("-------------------------------------------------------------------------");
+	}
+
+	/**
+	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
+	 */
+	protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder,
+												String kafkaHost,
+												String zookeeperConnectionString) throws Exception {
+		Properties kafkaProperties = new Properties();
+
+		// properties have to be Strings
+		kafkaProperties.put("advertised.host.name", kafkaHost);
+		kafkaProperties.put("broker.id", Integer.toString(brokerId));
+		kafkaProperties.put("log.dir", tmpFolder.toString());
+		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
+		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
+		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
+		
+		// for CI stability, increase zookeeper session timeout
+		kafkaProperties.put("zookeeper.session.timeout.ms", "20000");
+
+		final int numTries = 5;
+		
+		for (int i = 1; i <= numTries; i++) { 
+			int kafkaPort = NetUtils.getAvailablePort();
+			kafkaProperties.put("port", Integer.toString(kafkaPort));
+			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
+
+			try {
+				KafkaServer server = new KafkaServer(kafkaConfig, new KafkaLocalSystemTime());
+				server.startup();
+				return server;
+			}
+			catch (KafkaException e) {
+				if (e.getCause() instanceof BindException) {
+					// port conflict, retry...
+					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
+				}
+				else {
+					throw e;
+				}
+			}
+		}
+		
+		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
+	}
+
+	// ------------------------------------------------------------------------
+	//  Execution utilities
+	// ------------------------------------------------------------------------
+	
+	protected ZkClient createZookeeperClient() {
+		return new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
+	}
+	
+	protected static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
+		try {
+			see.execute(name);
+		}
+		catch (ProgramInvocationException | JobExecutionException root) {
+			Throwable cause = root.getCause();
+			
+			// search for nested SuccessExceptions
+			int depth = 0;
+			while (!(cause instanceof SuccessException)) {
+				if (cause == null || depth++ == 20) {
+					root.printStackTrace();
+					fail("Test failed: " + root.getMessage());
+				}
+				else {
+					cause = cause.getCause();
+				}
+			}
+		}
+	}
+
+	protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception {
+		try {
+			see.execute(name);
+		}
+		catch (ProgramInvocationException | JobExecutionException root) {
+			Throwable cause = root.getCause();
+
+			// search for nested SuccessExceptions
+			int depth = 0;
+			while (!(cause instanceof SuccessException)) {
+				if (cause == null || depth++ == 20) {
+					throw root;
+				}
+				else {
+					cause = cause.getCause();
+				}
+			}
+		}
+	}
+	
+	
+
+	protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
+		
+		// create topic with one client
+		Properties topicConfig = new Properties();
+		LOG.info("Creating topic {}", topic);
+
+		ZkClient creator = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
+		
+		AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig);
+		creator.close();
+		
+		// validate that the topic has been created
+		final long deadline = System.currentTimeMillis() + 30000;
+		do {
+			try {
+				Thread.sleep(100);
+			}
+			catch (InterruptedException e) {
+				// restore interrupted state
+			}
+			List<PartitionInfo> partitions = FlinkKafkaConsumer.getPartitionsForTopic(topic, standardProps);
+			if (partitions != null && partitions.size() > 0) {
+				return;
+			}
+		}
+		while (System.currentTimeMillis() < deadline);
+		fail ("Test topic could not be created");
+	}
+	
+	protected static void deleteTestTopic(String topic) {
+		LOG.info("Deleting topic {}", topic);
+
+		ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
+				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
+
+		AdminUtils.deleteTopic(zk, topic);
+		
+		zk.close();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java
new file mode 100644
index 0000000..75fdd46
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
+
+public class TestFixedPartitioner {
+
+
+	/**
+	 * <pre>
+	 *   		Flink Sinks:		Kafka Partitions
+	 * 			1	---------------->	1
+	 * 			2   --------------/
+	 * 			3   -------------/
+	 * 			4	------------/
+	 * </pre>
+	 */
+	@Test
+	public void testMoreFlinkThanBrokers() {
+		FixedPartitioner part = new FixedPartitioner();
+
+		int[] partitions = new int[]{0};
+
+		part.open(0, 4, partitions);
+		Assert.assertEquals(0, part.partition("abc1", partitions.length));
+
+		part.open(1, 4, partitions);
+		Assert.assertEquals(0, part.partition("abc2", partitions.length));
+
+		part.open(2, 4, partitions);
+		Assert.assertEquals(0, part.partition("abc3", partitions.length));
+		Assert.assertEquals(0, part.partition("abc3", partitions.length)); // check if it is changing ;)
+
+		part.open(3, 4, partitions);
+		Assert.assertEquals(0, part.partition("abc4", partitions.length));
+	}
+
+	/**
+	 *
+	 * <pre>
+	 * 		Flink Sinks:		Kafka Partitions
+	 * 			1	---------------->	1
+	 * 			2	---------------->	2
+	 * 									3
+	 * 									4
+	 * 									5
+	 *
+	 * </pre>
+	 */
+	@Test
+	public void testFewerPartitions() {
+		FixedPartitioner part = new FixedPartitioner();
+
+		int[] partitions = new int[]{0, 1, 2, 3, 4};
+		part.open(0, 2, partitions);
+		Assert.assertEquals(0, part.partition("abc1", partitions.length));
+		Assert.assertEquals(0, part.partition("abc1", partitions.length));
+
+		part.open(1, 2, partitions);
+		Assert.assertEquals(1, part.partition("abc1", partitions.length));
+		Assert.assertEquals(1, part.partition("abc1", partitions.length));
+	}
+
+	/*
+	 * 		Flink Sinks:		Kafka Partitions
+	 * 			1	------------>--->	1
+	 * 			2	-----------/----> 	2
+	 * 			3	----------/
+	 */
+	@Test
+	public void testMixedCase() {
+		FixedPartitioner part = new FixedPartitioner();
+		int[] partitions = new int[]{0,1};
+
+		part.open(0, 3, partitions);
+		Assert.assertEquals(0, part.partition("abc1", partitions.length));
+
+		part.open(1, 3, partitions);
+		Assert.assertEquals(1, part.partition("abc1", partitions.length));
+
+		part.open(2, 3, partitions);
+		Assert.assertEquals(0, part.partition("abc1", partitions.length));
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
new file mode 100644
index 0000000..27ad2e8
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import kafka.admin.AdminUtils;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.flink.streaming.connectors.kafka.KafkaTestBase;
+
+import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ZookeeperOffsetHandlerTest extends KafkaTestBase {
+	
+	@Test
+	public void runOffsetManipulationinZooKeeperTest() {
+		try {
+			final String topicName = "ZookeeperOffsetHandlerTest-Topic";
+			final String groupId = "ZookeeperOffsetHandlerTest-Group";
+			
+			final long offset = (long) (Math.random() * Long.MAX_VALUE);
+
+			ZkClient zkClient = createZookeeperClient();
+			AdminUtils.createTopic(zkClient, topicName, 3, 2, new Properties());
+				
+			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, groupId, topicName, 0, offset);
+	
+			long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, groupId, topicName, 0);
+
+			zkClient.close();
+			
+			assertEquals(offset, fetchedOffset);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
new file mode 100644
index 0000000..32377ae
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka.testutils;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+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.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
+import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
+
+import java.util.Random;
+
+@SuppressWarnings("serial")
+public class DataGenerators {
+	
+	public static void generateLongStringTupleSequence(StreamExecutionEnvironment env,
+														String brokerConnection, String topic,
+														int numPartitions,
+														final int from, final int to) throws Exception {
+
+		TypeInformation<Tuple2<Integer, Integer>> resultType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+
+		env.setParallelism(numPartitions);
+		env.getConfig().disableSysoutLogging();
+		env.setNumberOfExecutionRetries(0);
+		
+		DataStream<Tuple2<Integer, Integer>> stream =env.addSource(
+				new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
+
+					private volatile boolean running = true;
+
+					@Override
+					public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
+						int cnt = from;
+						int partition = getRuntimeContext().getIndexOfThisSubtask();
+
+						while (running && cnt <= to) {
+							ctx.collect(new Tuple2<Integer, Integer>(partition, cnt));
+							cnt++;
+						}
+					}
+
+					@Override
+					public void cancel() {
+						running = false;
+					}
+				});
+
+		stream.addSink(new FlinkKafkaProducer<>(topic,
+				new TypeInformationSerializationSchema<>(resultType, env.getConfig()),
+				FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection),
+				new Tuple2Partitioner(numPartitions)
+		));
+
+		env.execute("Data generator (Int, Int) stream to topic " + topic);
+	}
+
+	// ------------------------------------------------------------------------
+	
+	public static void generateRandomizedIntegerSequence(StreamExecutionEnvironment env,
+															String brokerConnection, String topic,
+															final int numPartitions,
+															final int numElements,
+															final boolean randomizeOrder) throws Exception {
+		env.setParallelism(numPartitions);
+		env.getConfig().disableSysoutLogging();
+		env.setNumberOfExecutionRetries(0);
+
+		DataStream<Integer> stream = env.addSource(
+				new RichParallelSourceFunction<Integer>() {
+
+					private volatile boolean running = true;
+
+					@Override
+					public void run(SourceContext<Integer> ctx) {
+						// create a sequence
+						int[] elements = new int[numElements];
+						for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask();
+								i < numElements;
+								i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) {
+							
+							elements[i] = val;
+						}
+
+						// scramble the sequence
+						if (randomizeOrder) {
+							Random rnd = new Random();
+							for (int i = 0; i < elements.length; i++) {
+								int otherPos = rnd.nextInt(elements.length);
+								
+								int tmp = elements[i];
+								elements[i] = elements[otherPos];
+								elements[otherPos] = tmp;
+							}
+						}
+
+						// emit the sequence
+						int pos = 0;
+						while (running && pos < elements.length) {
+							ctx.collect(elements[pos++]);
+						}
+					}
+
+					@Override
+					public void cancel() {
+						running = false;
+					}
+				});
+
+		stream
+				.rebalance()
+				.addSink(new FlinkKafkaProducer<>(topic,
+						new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig()),
+						FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection),
+						new KafkaPartitioner() {
+							@Override
+							public int partition(Object key, int numPartitions) {
+								return ((Integer) key) % numPartitions;
+							}
+						}));
+
+		env.execute("Scrambles int sequence generator");
+	}
+	
+	// ------------------------------------------------------------------------
+	
+	public static class InfiniteStringsGenerator extends Thread {
+
+		private final String kafkaConnectionString;
+		
+		private final String topic;
+		
+		private volatile Throwable error;
+		
+		private volatile boolean running = true;
+
+		
+		public InfiniteStringsGenerator(String kafkaConnectionString, String topic) {
+			this.kafkaConnectionString = kafkaConnectionString;
+			this.topic = topic;
+		}
+
+		@Override
+		public void run() {
+			// we manually feed data into the Kafka sink
+			FlinkKafkaProducer<String> producer = null;
+			try {
+				producer = new FlinkKafkaProducer<>(kafkaConnectionString, topic, new JavaDefaultStringSchema());
+				producer.setRuntimeContext(new MockRuntimeContext(1,0));
+				producer.open(new Configuration());
+				
+				final StringBuilder bld = new StringBuilder();
+				final Random rnd = new Random();
+				
+				while (running) {
+					bld.setLength(0);
+					
+					int len = rnd.nextInt(100) + 1;
+					for (int i = 0; i < len; i++) {
+						bld.append((char) (rnd.nextInt(20) + 'a') );
+					}
+					
+					String next = bld.toString();
+					producer.invoke(next);
+				}
+			}
+			catch (Throwable t) {
+				this.error = t;
+			}
+			finally {
+				if (producer != null) {
+					try {
+						producer.close();
+					}
+					catch (Throwable t) {
+						// ignore
+					}
+				}
+			}
+		}
+		
+		public void shutdown() {
+			this.running = false;
+			this.interrupt();
+		}
+		
+		public Throwable getError() {
+			return this.error;
+		}
+	}
+}


[02/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
new file mode 100644
index 0000000..d4a3a77
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
@@ -0,0 +1,499 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.functions.FoldFunction;
+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.aggregation.AggregationFunction;
+import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
+import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamGroupedFold;
+import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+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.windowing.evictors.CountEvictor;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+import org.apache.flink.streaming.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;
+
+/**
+ * A {@code KeyedStream} represents a {@link DataStream} on which operator state is
+ * partitioned by key using a provided {@link KeySelector}. Typical operations supported by a
+ * {@code DataStream} are also possible on a {@code KeyedStream}, with the exception of
+ * partitioning methods such as shuffle, forward and keyBy.
+ *
+ * <p>
+ * Reduce-style operations, such as {@link #reduce}, {@link #sum} and {@link #fold} work on elements
+ * that have the same key.
+ *
+ * @param <T> The type of the elements in the Keyed Stream.
+ * @param <KEY> The type of the key in the Keyed Stream.
+ */
+public class KeyedStream<T, KEY> extends DataStream<T> {
+
+	/** The key selector that can get the key by which the stream if partitioned from the elements */
+	private final KeySelector<T, KEY> keySelector;
+
+	/** The type of the key by which the stream is partitioned */
+	private final TypeInformation<KEY> keyType;
+	
+	/**
+	 * Creates a new {@link KeyedStream} using the given {@link KeySelector}
+	 * to partition operator state by key.
+	 * 
+	 * @param dataStream
+	 *            Base stream of data
+	 * @param keySelector
+	 *            Function for determining state partitions
+	 */
+	public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
+		this(dataStream, keySelector, TypeExtractor.getKeySelectorTypes(keySelector, dataStream.getType()));
+	}
+
+	/**
+	 * Creates a new {@link KeyedStream} using the given {@link KeySelector}
+	 * to partition operator state by key.
+	 *
+	 * @param dataStream
+	 *            Base stream of data
+	 * @param keySelector
+	 *            Function for determining state partitions
+	 */
+	public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector, TypeInformation<KEY> keyType) {
+		super(dataStream.getExecutionEnvironment(), new PartitionTransformation<>(
+				dataStream.getTransformation(), new HashPartitioner<>(keySelector)));
+		this.keySelector = keySelector;
+		this.keyType = keyType;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  properties
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the key selector that can get the key by which the stream if partitioned from the elements.
+	 * @return The key selector for the key.
+	 */
+	public KeySelector<T, KEY> getKeySelector() {
+		return this.keySelector;
+	}
+
+	/**
+	 * Gets the type of the key by which the stream is partitioned. 
+	 * @return The type of the key by which the stream is partitioned.
+	 */
+	public TypeInformation<KEY> getKeyType() {
+		return keyType;
+	}
+
+	@Override
+	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
+		throw new UnsupportedOperationException("Cannot override partitioning for KeyedStream.");
+	}
+
+	// ------------------------------------------------------------------------
+	//  basic transformations
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
+			TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
+
+		SingleOutputStreamOperator<R, ?> returnStream = super.transform(operatorName, outTypeInfo,operator);
+
+		// inject the key selector and key type
+		OneInputTransformation<T, R> transform = (OneInputTransformation<T, R>) returnStream.getTransformation();
+		transform.setStateKeySelector(keySelector);
+		transform.setStateKeyType(keyType);
+		
+		return returnStream;
+	}
+	
+	@Override
+	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
+		DataStreamSink<T> result = super.addSink(sinkFunction);
+		result.getTransformation().setStateKeySelector(keySelector);
+		result.getTransformation().setStateKeyType(keyType);
+		return result;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Windowing
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Windows this {@code KeyedStream} 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 WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size) {
+		return window(TumblingTimeWindows.of(size));
+	}
+
+	/**
+	 * Windows this {@code KeyedStream} 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 WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size, AbstractTime slide) {
+		return window(SlidingTimeWindows.of(size, slide));
+	}
+
+	/**
+	 * Windows this {@code KeyedStream} into tumbling count windows.
+	 *
+	 * @param size The size of the windows in number of elements.
+	 */
+	public WindowedStream<T, KEY, GlobalWindow> countWindow(long size) {
+		return window(GlobalWindows.create()).trigger(PurgingTrigger.of(CountTrigger.of(size)));
+	}
+
+	/**
+	 * Windows this {@code KeyedStream} into sliding count windows.
+	 *
+	 * @param size The size of the windows in number of elements.
+	 * @param slide The slide interval in number of elements.
+	 */
+	public WindowedStream<T, KEY, GlobalWindow> countWindow(long size, long slide) {
+		return window(GlobalWindows.create())
+				.evictor(CountEvictor.of(size))
+				.trigger(CountTrigger.of(slide));
+	}
+
+	/**
+	 * Windows this data stream to a {@code WindowedStream}, 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.
+	 *
+	 * <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.
+	 *
+	 * @param assigner The {@code WindowAssigner} that assigns elements to windows.
+	 * @return The trigger windows data stream.
+	 */
+	public <W extends Window> WindowedStream<T, KEY, W> window(WindowAssigner<? super T, W> assigner) {
+		return new WindowedStream<>(this, assigner);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Non-Windowed aggregation operations
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies a reduce transformation on the grouped data stream grouped on by
+	 * the given key position. The {@link ReduceFunction} will receive input
+	 * values based on the key value. Only input values with the same key will
+	 * go to the same reducer.
+	 *
+	 * @param reducer
+	 *            The {@link ReduceFunction} that will be called for every
+	 *            element of the input values with the same key.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> reducer) {
+		return transform("Keyed Reduce", getType(), new StreamGroupedReduce<T>(
+				clean(reducer), getType().createSerializer(getExecutionConfig())));
+	}
+
+	/**
+	 * Applies a fold transformation on the grouped data stream grouped on by
+	 * the given key position. The {@link FoldFunction} will receive input
+	 * values based on the key value. Only input values with the same key will
+	 * go to the same folder.
+	 *
+	 * @param folder
+	 *            The {@link FoldFunction} that will be called for every element
+	 *            of the input values with the same key.
+	 * @param initialValue
+	 *            The initialValue passed to the folders for each key.
+	 * @return The transformed DataStream.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> folder) {
+
+		TypeInformation<R> outType = TypeExtractor.getFoldReturnTypes(
+				clean(folder), getType(), Utils.getCallLocationName(), true);
+
+		return transform("Keyed Fold", outType, new StreamGroupedFold<>(clean(folder), initialValue));
+	}
+
+	/**
+	 * Applies an aggregation that gives a rolling sum of the data stream at the
+	 * given position grouped by the given key. An independent aggregate is kept
+	 * per key.
+	 *
+	 * @param positionToSum
+	 *            The position in the data point to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
+		return aggregate(new SumAggregator<>(positionToSum, getType(), getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current sum of the pojo data
+	 * stream at the given field expressionby the given key. An independent
+	 * aggregate is kept per key. A field expression is either the name of a
+	 * public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(String field) {
+		return aggregate(new SumAggregator<>(field, getType(), getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current minimum of the data
+	 * stream at the given position by the given key. An independent aggregate
+	 * is kept per key.
+	 *
+	 * @param positionToMin
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
+		return aggregate(new ComparableAggregator<>(positionToMin, getType(), AggregationFunction.AggregationType.MIN,
+				getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current minimum of the pojo
+	 * data stream at the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(String field) {
+		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MIN,
+				false, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the current maximum of the data stream
+	 * at the given position by the given key. An independent aggregate is kept
+	 * per key.
+	 *
+	 * @param positionToMax
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
+		return aggregate(new ComparableAggregator<>(positionToMax, getType(), AggregationFunction.AggregationType.MAX,
+				getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current maximum of the pojo
+	 * data stream at the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(String field) {
+		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAX,
+				false, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current minimum element of the
+	 * pojo data stream by the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @param first
+	 *            If True then in case of field equality the first object will
+	 *            be returned
+	 * @return The transformed DataStream.
+	 */
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator(field, getType(), AggregationFunction.AggregationType.MINBY,
+				first, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current maximum element of the
+	 * pojo data stream by the given field expression by the given key. An
+	 * independent aggregate is kept per key. A field expression is either the
+	 * name of a public field or a getter method with parentheses of the
+	 * {@link DataStream}S underlying type. A dot can be used to drill down into
+	 * objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field
+	 *            The field expression based on which the aggregation will be
+	 *            applied.
+	 * @param first
+	 *            If True then in case of field equality the first object will
+	 *            be returned
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAXBY,
+				first, getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the minimum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the minimum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * minimum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the minimum value at the
+	 * given position, the operator returns either the first or last one,
+	 * depending on the parameter set.
+	 *
+	 * @param positionToMinBy
+	 *            The position in the data point to minimize
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            minimal value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
+		return aggregate(new ComparableAggregator<T>(positionToMinBy, getType(), AggregationFunction.AggregationType.MINBY, first,
+				getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the maximum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the maximum value at the
+	 * given position, the operator returns the first one by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that that gives the current element with the
+	 * maximum value at the given position by the given key. An independent
+	 * aggregate is kept per key. If more elements have the maximum value at the
+	 * given position, the operator returns either the first or last one,
+	 * depending on the parameter set.
+	 *
+	 * @param positionToMaxBy
+	 *            The position in the data point to maximize.
+	 * @param first
+	 *            If true, then the operator return the first element with the
+	 *            maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMaxBy, getType(), AggregationFunction.AggregationType.MAXBY, first,
+				getExecutionConfig()));
+	}
+
+	protected SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregate) {
+		StreamGroupedReduce<T> operator = new StreamGroupedReduce<T>(
+				clean(aggregate), getType().createSerializer(getExecutionConfig()));
+		return transform("Keyed Aggregation", getType(), operator);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
new file mode 100644
index 0000000..33d5a3c
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.transformations.PartitionTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+
+/**
+ * The SingleOutputStreamOperator represents a user defined transformation
+ * applied on a {@link DataStream} with one predefined output type.
+ *
+ * @param <T> The type of the elements in this Stream
+ * @param <O> Type of the operator.
+ */
+public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<T, O>> extends DataStream<T> {
+
+	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, StreamTransformation<T> transformation) {
+		super(environment, transformation);
+	}
+
+	/**
+	 * Gets the name of the current data stream. This name is
+	 * used by the visualization and logging during runtime.
+	 *
+	 * @return Name of the stream.
+	 */
+	public String getName() {
+		return transformation.getName();
+	}
+
+	/**
+	 * Sets the name of the current data stream. This name is
+	 * used by the visualization and logging during runtime.
+	 *
+	 * @return The named operator.
+	 */
+	public SingleOutputStreamOperator<T, O> name(String name){
+		transformation.setName(name);
+		return this;
+	}
+
+	/**
+	 * Sets the parallelism for this operator. The degree must be 1 or more.
+	 * 
+	 * @param parallelism
+	 *            The parallelism for this operator.
+	 * @return The operator with set parallelism.
+	 */
+	public SingleOutputStreamOperator<T, O> setParallelism(int parallelism) {
+		if (parallelism < 1) {
+			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
+		}
+
+		transformation.setParallelism(parallelism);
+
+		return this;
+	}
+
+	/**
+	 * Sets the maximum time frequency (ms) for the flushing of the output
+	 * buffer. By default the output buffers flush only when they are full.
+	 * 
+	 * @param timeoutMillis
+	 *            The maximum time between two output flushes.
+	 * @return The operator with buffer timeout set.
+	 */
+	public SingleOutputStreamOperator<T, O> setBufferTimeout(long timeoutMillis) {
+		transformation.setBufferTimeout(timeoutMillis);
+		return this;
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<T, O> broadcast() {
+		return (SingleOutputStreamOperator<T, O>) super.broadcast();
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<T, O> shuffle() {
+		return (SingleOutputStreamOperator<T, O>) super.shuffle();
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<T, O> forward() {
+		return (SingleOutputStreamOperator<T, O>) super.forward();
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<T, O> rebalance() {
+		return (SingleOutputStreamOperator<T, O>) super.rebalance();
+	}
+
+	@SuppressWarnings("unchecked")
+	public SingleOutputStreamOperator<T, O> global() {
+		return (SingleOutputStreamOperator<T, O>) super.global();
+	}
+
+	/**
+	 * Sets the {@link ChainingStrategy} for the given operator affecting the
+	 * way operators will possibly be co-located on the same thread for
+	 * increased performance.
+	 * 
+	 * @param strategy
+	 *            The selected {@link ChainingStrategy}
+	 * @return The operator with the modified chaining strategy
+	 */
+	private SingleOutputStreamOperator<T, O> setChainingStrategy(ChainingStrategy strategy) {
+		this.transformation.setChainingStrategy(strategy);
+		return this;
+	}
+
+	/**
+	 * Turns off chaining for this operator so thread co-location will not be
+	 * used as an optimization. </p> Chaining can be turned off for the whole
+	 * job by {@link StreamExecutionEnvironment#disableOperatorChaining()}
+	 * however it is not advised for performance considerations.
+	 * 
+	 * @return The operator with chaining disabled
+	 */
+	public SingleOutputStreamOperator<T, O> disableChaining() {
+		return setChainingStrategy(ChainingStrategy.NEVER);
+	}
+
+	/**
+	 * Starts a new task chain beginning at this operator. This operator will
+	 * not be chained (thread co-located for increased performance) to any
+	 * previous tasks even if possible.
+	 * 
+	 * @return The operator with chaining set.
+	 */
+	public SingleOutputStreamOperator<T, O> startNewChain() {
+		return setChainingStrategy(ChainingStrategy.HEAD);
+	}
+
+	/**
+	 * Adds a type information hint about the return type of this operator. 
+	 * 
+	 * <p>
+	 * Type hints are important in cases where the Java compiler
+	 * throws away generic type information necessary for efficient execution.
+	 * 
+	 * <p>
+	 * This method takes a type information string that will be parsed. A type information string can contain the following
+	 * types:
+	 *
+	 * <ul>
+	 * <li>Basic types such as <code>Integer</code>, <code>String</code>, etc.
+	 * <li>Basic type arrays such as <code>Integer[]</code>,
+	 * <code>String[]</code>, etc.
+	 * <li>Tuple types such as <code>Tuple1&lt;TYPE0&gt;</code>,
+	 * <code>Tuple2&lt;TYPE0, TYPE1&gt;</code>, etc.</li>
+	 * <li>Pojo types such as <code>org.my.MyPojo&lt;myFieldName=TYPE0,myFieldName2=TYPE1&gt;</code>, etc.</li>
+	 * <li>Generic types such as <code>java.lang.Class</code>, etc.
+	 * <li>Custom type arrays such as <code>org.my.CustomClass[]</code>,
+	 * <code>org.my.CustomClass$StaticInnerClass[]</code>, etc.
+	 * <li>Value types such as <code>DoubleValue</code>,
+	 * <code>StringValue</code>, <code>IntegerValue</code>, etc.</li>
+	 * <li>Tuple array types such as <code>Tuple2&lt;TYPE0,TYPE1&gt;[], etc.</code></li>
+	 * <li>Writable types such as <code>Writable&lt;org.my.CustomWritable&gt;</code></li>
+	 * <li>Enum types such as <code>Enum&lt;org.my.CustomEnum&gt;</code></li>
+	 * </ul>
+	 *
+	 * Example:
+	 * <code>"Tuple2&lt;String,Tuple2&lt;Integer,org.my.MyJob$Pojo&lt;word=String&gt;&gt;&gt;"</code>
+	 *
+	 * @param typeInfoString
+	 *            type information string to be parsed
+	 * @return This operator with a given return type hint.
+	 */
+	public O returns(String typeInfoString) {
+		if (typeInfoString == null) {
+			throw new IllegalArgumentException("Type information string must not be null.");
+		}
+		return returns(TypeInfoParser.<T>parse(typeInfoString));
+	}
+	
+	/**
+	 * Adds a type information hint about the return type of this operator. 
+	 * 
+	 * <p>
+	 * Type hints are important in cases where the Java compiler
+	 * throws away generic type information necessary for efficient execution.
+	 * 
+	 * <p>
+	 * This method takes an instance of {@link org.apache.flink.api.common.typeinfo.TypeInformation} such as:
+	 * 
+	 * <ul>
+	 * <li>{@link org.apache.flink.api.common.typeinfo.BasicTypeInfo}</li>
+	 * <li>{@link org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo}</li>
+	 * <li>{@link org.apache.flink.api.java.typeutils.TupleTypeInfo}</li>
+	 * <li>{@link org.apache.flink.api.java.typeutils.PojoTypeInfo}</li>
+	 * <li>{@link org.apache.flink.api.java.typeutils.WritableTypeInfo}</li>
+	 * <li>{@link org.apache.flink.api.java.typeutils.ValueTypeInfo}</li>
+	 * <li>etc.</li>
+	 * </ul>
+	 *
+	 * @param typeInfo
+	 *            type information as a return type hint
+	 * @return This operator with a given return type hint.
+	 */
+	public O returns(TypeInformation<T> typeInfo) {
+		if (typeInfo == null) {
+			throw new IllegalArgumentException("Type information must not be null.");
+		}
+		transformation.setOutputType(typeInfo);
+		@SuppressWarnings("unchecked")
+		O returnType = (O) this;
+		return returnType;
+	}
+	
+	/**
+	 * Adds a type information hint about the return type of this operator. 
+	 * 
+	 * <p>
+	 * Type hints are important in cases where the Java compiler
+	 * throws away generic type information necessary for efficient execution.
+	 * 
+	 * <p>
+	 * This method takes a class that will be analyzed by Flink's type extraction capabilities.
+	 * 
+	 * <p>
+	 * Examples for classes are:
+	 * <ul>
+	 * <li>Basic types such as <code>Integer.class</code>, <code>String.class</code>, etc.</li>
+	 * <li>POJOs such as <code>MyPojo.class</code></li>
+	 * <li>Classes that <b>extend</b> tuples. Classes like <code>Tuple1.class</code>,<code>Tuple2.class</code>, etc. are <b>not</b> sufficient.</li>
+	 * <li>Arrays such as <code>String[].class</code>, etc.</li>
+	 * </ul>
+	 *
+	 * @param typeClass
+	 *            class as a return type hint
+	 * @return This operator with a given return type hint.
+	 */
+	@SuppressWarnings("unchecked")
+	public O returns(Class<T> typeClass) {
+		if (typeClass == null) {
+			throw new IllegalArgumentException("Type class must not be null.");
+		}
+		
+		try {
+			TypeInformation<T> ti = (TypeInformation<T>) TypeExtractor.createTypeInfo(typeClass);
+			return returns(ti);
+		}
+		catch (InvalidTypesException e) {
+			throw new InvalidTypesException("The given class is not suited for providing necessary type information.", e);
+		}
+	}
+
+	@Override
+	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
+		return new SingleOutputStreamOperator<T, O>(this.getExecutionEnvironment(), new PartitionTransformation<T>(this.getTransformation(), partitioner));
+	}
+
+	/**
+	 * By default all operators in a streaming job share the same resource
+	 * group. Each resource group takes as many task manager slots as the
+	 * maximum parallelism operator in that group. Task chaining is only
+	 * possible within one resource group. By calling this method, this
+	 * operators starts a new resource group and all subsequent operators will
+	 * be added to this group unless specified otherwise. </p> Please note that
+	 * local executions have by default as many available task slots as the
+	 * environment parallelism, so in order to start a new resource group the
+	 * degree of parallelism for the operators must be decreased from the
+	 * default.
+	 * 
+	 * @return The operator as a part of a new resource group.
+	 */
+	public SingleOutputStreamOperator<T, O> startNewResourceGroup() {
+		transformation.setResourceStrategy(ResourceStrategy.NEWGROUP);
+		return this;
+	}
+
+	/**
+	 * Isolates the operator in its own resource group. This will cause the
+	 * operator to grab as many task slots as its degree of parallelism. If
+	 * there are no free resources available, the job will fail to start. It
+	 * also disables chaining for this operator </p>All subsequent operators are
+	 * assigned to the default resource group.
+	 * 
+	 * @return The operator with isolated resource group.
+	 */
+	public SingleOutputStreamOperator<T, O> isolateResources() {
+		transformation.setResourceStrategy(ResourceStrategy.ISOLATE);
+		return this;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
new file mode 100644
index 0000000..11ee7f2
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.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.datastream;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.transformations.SelectTransformation;
+import org.apache.flink.streaming.api.transformations.SplitTransformation;
+
+/**
+ * The SplitStream represents an operator that has been split using an
+ * {@link OutputSelector}. Named outputs can be selected using the
+ * {@link #select} function. To apply transformation on the whole output simply
+ * call the transformation on the SplitStream
+ *
+ * @param <OUT> The type of the elements in the Stream
+ */
+public class SplitStream<OUT> extends DataStream<OUT> {
+
+	protected SplitStream(DataStream<OUT> dataStream, OutputSelector<OUT> outputSelector) {
+		super(dataStream.getExecutionEnvironment(), new SplitTransformation<OUT>(dataStream.getTransformation(), outputSelector));
+	}
+
+	/**
+	 * Sets the output names for which the next operator will receive values.
+	 * 
+	 * @param outputNames
+	 *            The output names for which the operator will receive the
+	 *            input.
+	 * @return Returns the selected DataStream
+	 */
+	public DataStream<OUT> select(String... outputNames) {
+		return selectOutput(outputNames);
+	}
+
+	private DataStream<OUT> selectOutput(String[] outputNames) {
+		for (String outName : outputNames) {
+			if (outName == null) {
+				throw new RuntimeException("Selected names must not be null");
+			}
+		}
+
+		SelectTransformation<OUT> selectTransform = new SelectTransformation<OUT>(this.getTransformation(), Lists.newArrayList(outputNames));
+		return new DataStream<OUT>(this.getExecutionEnvironment(), selectTransform);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
new file mode 100644
index 0000000..149d7a8
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
@@ -0,0 +1,484 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple10;
+import org.apache.flink.api.java.tuple.Tuple11;
+import org.apache.flink.api.java.tuple.Tuple12;
+import org.apache.flink.api.java.tuple.Tuple13;
+import org.apache.flink.api.java.tuple.Tuple14;
+import org.apache.flink.api.java.tuple.Tuple15;
+import org.apache.flink.api.java.tuple.Tuple16;
+import org.apache.flink.api.java.tuple.Tuple17;
+import org.apache.flink.api.java.tuple.Tuple18;
+import org.apache.flink.api.java.tuple.Tuple19;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple20;
+import org.apache.flink.api.java.tuple.Tuple21;
+import org.apache.flink.api.java.tuple.Tuple22;
+import org.apache.flink.api.java.tuple.Tuple23;
+import org.apache.flink.api.java.tuple.Tuple24;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.api.java.tuple.Tuple8;
+import org.apache.flink.api.java.tuple.Tuple9;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.streaming.api.operators.StreamProject;
+
+import com.google.common.base.Preconditions;
+
+public class StreamProjection<IN> {
+
+	private DataStream<IN> dataStream;
+	private int[] fieldIndexes;
+
+	protected StreamProjection(DataStream<IN> dataStream, int[] fieldIndexes) {
+		if (!dataStream.getType().isTupleType()) {
+			throw new RuntimeException("Only Tuple DataStreams can be projected");
+		}
+		if(fieldIndexes.length == 0) {
+			throw new IllegalArgumentException("project() needs to select at least one (1) field.");
+		} else if(fieldIndexes.length > Tuple.MAX_ARITY - 1) {
+			throw new IllegalArgumentException(
+					"project() may select only up to (" + (Tuple.MAX_ARITY - 1) + ") fields.");
+		}
+
+		int maxFieldIndex = (dataStream.getType()).getArity();
+		for(int i = 0; i < fieldIndexes.length; i++) {
+			Preconditions.checkElementIndex(fieldIndexes[i], maxFieldIndex);
+		}
+
+		this.dataStream = dataStream;
+		this.fieldIndexes = fieldIndexes;
+	}
+
+	/**
+	 * Chooses a projectTupleX according to the length of
+	 * {@link org.apache.flink.streaming.api.datastream.StreamProjection#fieldIndexes}
+	 *
+	 * @return The projected DataStream.
+	 * @see org.apache.flink.api.java.operators.ProjectOperator.Projection
+	 */
+	@SuppressWarnings("unchecked")
+	public <OUT extends Tuple> SingleOutputStreamOperator<OUT, ?> projectTupleX() {
+		SingleOutputStreamOperator<OUT, ?> projOperator = null;
+
+		switch (fieldIndexes.length) {
+			case 1: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple1(); break;
+			case 2: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple2(); break;
+			case 3: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple3(); break;
+			case 4: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple4(); break;
+			case 5: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple5(); break;
+			case 6: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple6(); break;
+			case 7: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple7(); break;
+			case 8: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple8(); break;
+			case 9: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple9(); break;
+			case 10: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple10(); break;
+			case 11: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple11(); break;
+			case 12: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple12(); break;
+			case 13: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple13(); break;
+			case 14: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple14(); break;
+			case 15: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple15(); break;
+			case 16: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple16(); break;
+			case 17: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple17(); break;
+			case 18: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple18(); break;
+			case 19: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple19(); break;
+			case 20: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple20(); break;
+			case 21: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple21(); break;
+			case 22: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple22(); break;
+			case 23: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple23(); break;
+			case 24: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple24(); break;
+			case 25: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple25(); break;
+			default:
+				throw new IllegalStateException("Excessive arity in tuple.");
+		}
+
+		return projOperator;
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0> SingleOutputStreamOperator<Tuple1<T0>, ?> projectTuple1() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple1<T0>> tType = new TupleTypeInfo<Tuple1<T0>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple1<T0>>(
+				fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1> SingleOutputStreamOperator<Tuple2<T0, T1>, ?> projectTuple2() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple2<T0, T1>> tType = new TupleTypeInfo<Tuple2<T0, T1>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple2<T0, T1>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2> SingleOutputStreamOperator<Tuple3<T0, T1, T2>, ?> projectTuple3() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple3<T0, T1, T2>> tType = new TupleTypeInfo<Tuple3<T0, T1, T2>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple3<T0, T1, T2>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3> SingleOutputStreamOperator<Tuple4<T0, T1, T2, T3>, ?> projectTuple4() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple4<T0, T1, T2, T3>> tType = new TupleTypeInfo<Tuple4<T0, T1, T2, T3>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple4<T0, T1, T2, T3>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4> SingleOutputStreamOperator<Tuple5<T0, T1, T2, T3, T4>, ?> projectTuple5() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>> tType = new TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple5<T0, T1, T2, T3, T4>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5> SingleOutputStreamOperator<Tuple6<T0, T1, T2, T3, T4, T5>, ?> projectTuple6() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>> tType = new TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple6<T0, T1, T2, T3, T4, T5>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6> SingleOutputStreamOperator<Tuple7<T0, T1, T2, T3, T4, T5, T6>, ?> projectTuple7() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>> tType = new TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7> SingleOutputStreamOperator<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>, ?> projectTuple8() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> tType = new TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8> SingleOutputStreamOperator<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>, ?> projectTuple9() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> tType = new TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> SingleOutputStreamOperator<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>, ?> projectTuple10() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> tType = new TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> SingleOutputStreamOperator<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>, ?> projectTuple11() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> tType = new TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> SingleOutputStreamOperator<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>, ?> projectTuple12() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> tType = new TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> SingleOutputStreamOperator<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>, ?> projectTuple13() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> tType = new TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13> SingleOutputStreamOperator<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>, ?> projectTuple14() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> tType = new TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14> SingleOutputStreamOperator<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>, ?> projectTuple15() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> tType = new TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15> SingleOutputStreamOperator<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>, ?> projectTuple16() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> tType = new TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16> SingleOutputStreamOperator<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>, ?> projectTuple17() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> tType = new TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17> SingleOutputStreamOperator<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>, ?> projectTuple18() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> tType = new TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18> SingleOutputStreamOperator<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>, ?> projectTuple19() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> tType = new TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19> SingleOutputStreamOperator<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>, ?> projectTuple20() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> tType = new TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20> SingleOutputStreamOperator<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>, ?> projectTuple21() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> tType = new TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21> SingleOutputStreamOperator<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>, ?> projectTuple22() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> tType = new TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22> SingleOutputStreamOperator<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>, ?> projectTuple23() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> tType = new TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23> SingleOutputStreamOperator<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>, ?> projectTuple24() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>> tType = new TupleTypeInfo<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	/**
+	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
+	 *
+	 * @return The projected DataStream.
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24> SingleOutputStreamOperator<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>, ?> projectTuple25() {
+		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
+		TupleTypeInfo<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>> tType = new TupleTypeInfo<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(fTypes);
+
+		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
+	}
+
+	public static TypeInformation<?>[] extractFieldTypes(int[] fields, TypeInformation<?> inType) {
+
+		TupleTypeInfo<?> inTupleType = (TupleTypeInfo<?>) inType;
+		TypeInformation<?>[] fieldTypes = new TypeInformation[fields.length];
+
+		for (int i = 0; i < fields.length; i++) {
+			fieldTypes[i] = inTupleType.getTypeAt(fields[i]);
+		}
+
+		return fieldTypes;
+	}
+
+}


[50/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
deleted file mode 100644
index 04ae40a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.elasticsearch;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.elasticsearch.action.index.IndexRequest;
-
-import java.io.Serializable;
-
-/**
- * Function that creates an {@link IndexRequest} from an element in a Stream.
- *
- * <p>
- * This is used by {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink}
- * to prepare elements for sending them to Elasticsearch. See
- * <a href="https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/index_.html">Index API</a>
- * for information about how to format data for adding it to an Elasticsearch index.
- *
- * <p>
- * Example:
- *
- * <pre>{@code
- *     private static class MyIndexRequestBuilder implements IndexRequestBuilder<String> {
- *
- *         public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
- *             Map<String, Object> json = new HashMap<>();
- *             json.put("data", element);
- *
- *             return Requests.indexRequest()
- *                 .index("my-index")
- *                 .type("my-type")
- *                 .source(json);
- *         }
- *     }
- * }</pre>
- *
- * @param <T> The type of the element handled by this {@code IndexRequestBuilder}
- */
-public interface IndexRequestBuilder<T> extends Function, Serializable {
-
-	/**
-	 * Creates an {@link org.elasticsearch.action.index.IndexRequest} from an element.
-	 *
-	 * @param element The element that needs to be turned in to an {@code IndexRequest}
-	 * @param ctx The Flink {@link RuntimeContext} of the {@link ElasticsearchSink}
-	 *
-	 * @return The constructed {@code IndexRequest}
-	 */
-	IndexRequest createIndexRequest(T element, RuntimeContext ctx);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
deleted file mode 100644
index 298eb64..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.elasticsearch.examples;
-
-import com.google.common.collect.Maps;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink;
-import org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.client.Requests;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that
- * you have a cluster names "elasticsearch" running or change the cluster name in the config map.
- */
-public class ElasticsearchExample {
-
-	public static void main(String[] args) throws Exception {
-		
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<String> source = env.addSource(new SourceFunction<String>() {
-			private static final long serialVersionUID = 1L;
-
-			private volatile boolean running = true;
-
-			@Override
-			public void run(SourceContext<String> ctx) throws Exception {
-				for (int i = 0; i < 20 && running; i++) {
-					ctx.collect("message #" + i);
-				}
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		});
-
-		Map<String, String> config = Maps.newHashMap();
-		// This instructs the sink to emit after every element, otherwise they would be buffered
-		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-
-		source.addSink(new ElasticsearchSink<>(config, new IndexRequestBuilder<String>() {
-			@Override
-			public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
-				Map<String, Object> json = new HashMap<>();
-				json.put("data", element);
-
-				return Requests.indexRequest()
-						.index("my-index")
-						.type("my-type")
-						.source(json);
-			}
-		}));
-
-
-		env.execute("Elasticsearch Example");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
deleted file mode 100644
index 33a2e47..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
+++ /dev/null
@@ -1,205 +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.connectors.elasticsearch;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.elasticsearch.action.get.GetRequest;
-import org.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.client.Requests;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.transport.LocalTransportAddress;
-import org.elasticsearch.common.transport.TransportAddress;
-import org.elasticsearch.node.Node;
-import org.junit.Assert;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
-
-public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
-
-	private static final int NUM_ELEMENTS = 20;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Test
-	public void testNodeClient() throws Exception{
-
-		File dataDir = tempFolder.newFolder();
-
-		Node node = nodeBuilder()
-				.settings(ImmutableSettings.settingsBuilder()
-						.put("http.enabled", false)
-						.put("path.data", dataDir.getAbsolutePath()))
-				// set a custom cluster name to verify that user config works correctly
-				.clusterName("my-node-client-cluster")
-				.local(true)
-				.node();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
-
-		Map<String, String> config = Maps.newHashMap();
-		// This instructs the sink to emit after every element, otherwise they would be buffered
-		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-		config.put("cluster.name", "my-node-client-cluster");
-
-		// connect to our local node
-		config.put("node.local", "true");
-
-		source.addSink(new ElasticsearchSink<>(config, new TestIndexRequestBuilder()));
-
-		env.execute("Elasticsearch Node Client Test");
-
-
-		// verify the results
-		Client client = node.client();
-		for (int i = 0; i < NUM_ELEMENTS; i++) {
-			GetResponse response = client.get(new GetRequest("my-index",
-					"my-type",
-					Integer.toString(i))).actionGet();
-			Assert.assertEquals("message #" + i, response.getSource().get("data"));
-		}
-
-		node.close();
-	}
-
-	@Test
-	public void testTransportClient() throws Exception {
-
-		File dataDir = tempFolder.newFolder();
-
-		Node node = nodeBuilder()
-				.settings(ImmutableSettings.settingsBuilder()
-						.put("http.enabled", false)
-						.put("path.data", dataDir.getAbsolutePath()))
-						// set a custom cluster name to verify that user config works correctly
-				.clusterName("my-node-client-cluster")
-				.local(true)
-				.node();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
-
-		Map<String, String> config = Maps.newHashMap();
-		// This instructs the sink to emit after every element, otherwise they would be buffered
-		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-		config.put("cluster.name", "my-node-client-cluster");
-
-		// connect to our local node
-		config.put("node.local", "true");
-
-		List<TransportAddress> transports = Lists.newArrayList();
-		transports.add(new LocalTransportAddress("1"));
-
-		source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
-
-		env.execute("Elasticsearch TransportClient Test");
-
-
-		// verify the results
-		Client client = node.client();
-		for (int i = 0; i < NUM_ELEMENTS; i++) {
-			GetResponse response = client.get(new GetRequest("my-index",
-					"my-type",
-					Integer.toString(i))).actionGet();
-			Assert.assertEquals("message #" + i, response.getSource().get("data"));
-		}
-
-		node.close();
-	}
-
-	@Test(expected = JobExecutionException.class)
-	public void testTransportClientFails() throws Exception{
-		// this checks whether the TransportClient fails early when there is no cluster to
-		// connect to. We don't hava such as test for the Node Client version since that
-		// one will block and wait for a cluster to come online
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
-
-		Map<String, String> config = Maps.newHashMap();
-		// This instructs the sink to emit after every element, otherwise they would be buffered
-		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-		config.put("cluster.name", "my-node-client-cluster");
-
-		// connect to our local node
-		config.put("node.local", "true");
-
-		List<TransportAddress> transports = Lists.newArrayList();
-		transports.add(new LocalTransportAddress("1"));
-
-		source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
-
-		env.execute("Elasticsearch Node Client Test");
-	}
-
-	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean running = true;
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
-			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
-				ctx.collect(Tuple2.of(i, "message #" + i));
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-
-	private static class TestIndexRequestBuilder implements IndexRequestBuilder<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public IndexRequest createIndexRequest(Tuple2<Integer, String> element, RuntimeContext ctx) {
-			Map<String, Object> json = new HashMap<>();
-			json.put("data", element.f1);
-
-			return Requests.indexRequest()
-					.index("my-index")
-					.type("my-type")
-					.id(element.f0.toString())
-					.source(json);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
deleted file mode 100644
index dc20726..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-log4j.rootLogger=OFF, testlogger
-
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
-
-# suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml
deleted file mode 100644
index 45b3b92..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +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.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.streaming" level="WARN"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/pom.xml
deleted file mode 100644
index e0319b1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/pom.xml
+++ /dev/null
@@ -1,112 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-filesystem</artifactId>
-	<name>flink-connector-filesystem</name>
-
-	<packaging>jar</packaging>
-
-	<!--
-		This is a Hadoop2 only flink module.
-	-->
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>${shading-artifact.name}</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime</artifactId>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
-		</dependency>
-
-	</dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
deleted file mode 100644
index 913da97..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.hadoop.fs.Path;
-
-import java.io.Serializable;
-
-/**
- * A bucketer is used with a {@link RollingSink}
- * to put emitted elements into rolling files.
- *
- * <p>
- * The {@code RollingSink} has one active bucket that it is writing to at a time. Whenever
- * a new element arrives it will ask the {@code Bucketer} if a new bucket should be started and
- * the old one closed. The {@code Bucketer} can, for example, decide to start new buckets
- * based on system time.
- */
-public interface Bucketer extends Serializable {
-
-	/**
-	 * Returns {@code true} when a new bucket should be started.
-	 *
-	 * @param currentBucketPath The bucket {@code Path} that is currently being used.
-	 */
-	boolean shouldStartNewBucket(Path basePath, Path currentBucketPath);
-
-	/**
-	 * Returns the {@link Path} of a new bucket file.
-	 *
-	 * @param basePath The base path containing all the buckets.
-	 *
-	 * @return The complete new {@code Path} of the new bucket. This should include the {@code basePath}
-	 *      and also the {@code subtaskIndex} tp avoid clashes with parallel sinks.
-	 */
-	Path getNextBucketPath(Path basePath);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java
deleted file mode 100644
index 152c75a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs;
-
-/**
- * A clock that can provide the current time.
- *
- * <p>
- * Normally this would be system time, but for testing a custom {@code Clock} can be provided.
- */
-public interface Clock {
-
-	/**
-	 * Return the current system time in milliseconds.
-	 */
-	public long currentTimeMillis();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java
deleted file mode 100644
index 0be40f5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-/**
- * A {@link Bucketer} that assigns to buckets based on current system time.
- *
- * <p>
- * The {@code DateTimeBucketer} will create directories of the following form:
- * {@code /{basePath}/{dateTimePath}/}. The {@code basePath} is the path
- * that was specified as a base path when creating the
- * {@link RollingSink}. The {@code dateTimePath}
- * is determined based on the current system time and the user provided format string.
- *
- * <p>
- * {@link SimpleDateFormat} is used to derive a date string from the current system time and
- * the date format string. The default format string is {@code "yyyy-MM-dd--HH"} so the rolling
- * files will have a granularity of hours.
- *
- *
- * <p>
- * Example:
- *
- * <pre>{@code
- *     Bucketer buck = new DateTimeBucketer("yyyy-MM-dd--HH");
- * }</pre>
- *
- * This will create for example the following bucket path:
- * {@code /base/1976-12-31-14/}
- *
- */
-public class DateTimeBucketer implements Bucketer {
-
-	private static Logger LOG = LoggerFactory.getLogger(DateTimeBucketer.class);
-
-	private static final long serialVersionUID = 1L;
-
-	private static final String DEFAULT_FORMAT_STRING = "yyyy-MM-dd--HH";
-
-	// We have this so that we can manually set it for tests.
-	private static Clock clock = new SystemClock();
-
-	private final String formatString;
-
-	private transient SimpleDateFormat dateFormatter;
-
-	/**
-	 * Creates a new {@code DateTimeBucketer} with format string {@code "yyyy-MM-dd--HH"}.
-	 */
-	public DateTimeBucketer() {
-		this(DEFAULT_FORMAT_STRING);
-	}
-
-	/**
-	 * Creates a new {@code DateTimeBucketer} with the given date/time format string.
-	 *
-	 * @param formatString The format string that will be given to {@code SimpleDateFormat} to determine
-	 *                     the bucket path.
-	 */
-	public DateTimeBucketer(String formatString) {
-		this.formatString = formatString;
-
-		this.dateFormatter = new SimpleDateFormat(formatString);
-	}
-
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-
-		this.dateFormatter = new SimpleDateFormat(formatString);
-	}
-
-
-	@Override
-	public boolean shouldStartNewBucket(Path basePath, Path currentBucketPath) {
-		String newDateTimeString = dateFormatter.format(new Date(clock.currentTimeMillis()));
-		return !(new Path(basePath, newDateTimeString).equals(currentBucketPath));
-	}
-
-	@Override
-	public Path getNextBucketPath(Path basePath) {
-		String newDateTimeString = dateFormatter.format(new Date(clock.currentTimeMillis()));
-		return new Path(basePath + "/" + newDateTimeString);
-	}
-
-	@Override
-	public String toString() {
-		return "DateTimeBucketer{" +
-				"formatString='" + formatString + '\'' +
-				'}';
-	}
-
-	/**
-	 * This sets the internal {@link Clock} implementation. This method should only be used for testing
-	 *
-	 * @param newClock The new clock to set.
-	 */
-	public static void setClock(Clock newClock) {
-		clock = newClock;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java
deleted file mode 100644
index 1307d11..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.fs;
-
-import org.apache.hadoop.fs.Path;
-
-/**
- * A {@link org.apache.flink.streaming.connectors.fs.Bucketer} that does not perform any
- * rolling of files. All files are written to the base path.
- */
-public class NonRollingBucketer implements Bucketer {
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public boolean shouldStartNewBucket(Path basePath, Path currentBucketPath) {
-		return false;
-	}
-
-	@Override
-	public Path getNextBucketPath(Path basePath) {
-		return basePath;
-	}
-
-	@Override
-	public String toString() {
-		return "NonRollingBucketer";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
deleted file mode 100644
index c705767..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
+++ /dev/null
@@ -1,900 +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.connectors.fs;
-
-import com.google.common.collect.Lists;
-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.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-/**
- * Sink that emits its input elements to rolling {@link org.apache.hadoop.fs.FileSystem} files. This
- * is itegrated with the checkpointing mechanism to provide exactly once semantics.
- *
- * <p>
- * When creating the sink a {@code basePath} must be specified. The base directory contains
- * one directory for every bucket. The bucket directories themselves contain several part files.
- * These contain the actual written data.
- *
- * <p>
- * The sink uses a {@link Bucketer} to determine the name of bucket directories inside the
- * base directory. Whenever the {@code Bucketer} returns a different directory name than
- * it returned before the sink will close the current part files inside that bucket
- * and start the new bucket directory. The default bucketer is a {@link DateTimeBucketer} with
- * date format string {@code ""yyyy-MM-dd--HH"}. You can specify a custom {@code Bucketer}
- * using {@link #setBucketer(Bucketer)}. For example, use
- * {@link org.apache.flink.streaming.connectors.fs.NonRollingBucketer} if you don't want to have
- * buckets but still write part files in a fault-tolerant way.
- *
- * <p>
- * The filenames of the part files contain the part prefix, the parallel subtask index of the sink
- * and a rolling counter, for example {@code "part-1-17"}. Per default the part prefix is
- * {@code "part"} but this can be
- * configured using {@link #setPartPrefix(String)}. When a part file becomes bigger
- * than the batch size the current part file is closed, the part counter is increased and
- * a new part file is created. The batch size defaults to {@code 384MB}, this can be configured
- * using {@link #setBatchSize(long)}.
- *
- * <p>
- * Part files can be in one of three states: in-progress, pending or finished. The reason for this
- * is how the sink works together with the checkpointing mechanism to provide exactly-once semantics
- * and fault-tolerance. The part file that is currently being written to is in-progress. Once
- * a part file is closed for writing it becomes pending. When a checkpoint is successful the
- * currently pending files will be moved to finished. If a failure occurs the pending files
- * will be deleted to reset state to the last checkpoint. The data in in-progress files will
- * also have to be rolled back. If the {@code FileSystem} supports the {@code truncate} call
- * this will be used to reset the file back to a previous state. If not, a special file
- * with the same name as the part file and the suffix {@code ".valid-length"} will be written
- * that contains the length up to which the file contains valid data. When reading the file
- * it must be ensured that it is only read up to that point. The prefixes and suffixes for
- * the different file states and valid-length files can be configured, for example with
- * {@link #setPendingSuffix(String)}.
- *
- * <p>
- * Note: If checkpointing is not enabled the pending files will never be moved to the finished state.
- * In that case, the pending suffix/prefix can be set to {@code ""} to make the sink work
- * in a non-fault-tolerant way but still provide output without prefixes and suffixes.
- *
- * <p>
- * The part files are written using an instance of {@link Writer}. By default
- * {@link org.apache.flink.streaming.connectors.fs.StringWriter} is used, which writes the result
- * of {@code toString()} for every element. Separated by newlines. You can configure the writer
- * using {@link #setWriter(Writer)}. For example,
- * {@link org.apache.flink.streaming.connectors.fs.SequenceFileWriter} can be used to write
- * Hadoop {@code SequenceFiles}.
- *
- * <p>
- * Example:
- *
- * <pre>{@code
- *     new RollingSink<Tuple2<IntWritable, Text>>(outPath)
- *         .setWriter(new SequenceFileWriter<IntWritable, Text>())
- *         .setBucketer(new DateTimeBucketer("yyyy-MM-dd--HHmm")
- * }</pre>
- *
- * This will create a sink that writes to {@code SequenceFiles} and rolls every minute.
- *
- * @see org.apache.flink.streaming.connectors.fs.DateTimeBucketer
- * @see StringWriter
- * @see SequenceFileWriter
- *
- * @param <T> Type of the elements emitted by this sink
- */
-public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConfigurable, Checkpointed<RollingSink.BucketState>, CheckpointNotifier {
-	private static final long serialVersionUID = 1L;
-
-	private static Logger LOG = LoggerFactory.getLogger(RollingSink.class);
-
-
-	// --------------------------------------------------------------------------------------------
-	//  User configuration values
-	// --------------------------------------------------------------------------------------------
-	// These are initialized with some defaults but are meant to be changeable by the user
-
-	/**
-	 * The default maximum size of part files.
-	 *
-	 * 6 times the default block size
-	 */
-	private final long DEFAULT_BATCH_SIZE = 1024L * 1024L * 384L;
-
-	/**
-	 * This is used for part files that we are writing to but which where not yet confirmed
-	 * by a checkpoint.
-	 */
-	private final String DEFAULT_IN_PROGRESS_SUFFIX = ".in-progress";
-
-	/**
-	 * See above, but for prefix
-	 */
-	private final String DEFAULT_IN_PROGRESS_PREFIX = "_";
-
-	/**
-	 * This is used for part files that we are not writing to but which are not yet confirmed by
-	 * checkpoint.
-	 */
-	private final String DEFAULT_PENDING_SUFFIX = ".pending";
-
-	/**
-	 * See above, but for prefix.
-	 */
-	private final String DEFAULT_PENDING_PREFIX = "_";
-
-	/**
-	 * When truncate() is not supported on the used FileSystem we instead write a
-	 * file along the part file with this ending that contains the length up to which
-	 * the part file is valid.
-	 */
-	private final String DEFAULT_VALID_SUFFIX = ".valid-length";
-
-	/**
-	 * See above, but for prefix.
-	 */
-	private final String DEFAULT_VALID_PREFIX = "_";
-
-	/**
-	 * The default prefix for part files.
-	 */
-	private final String DEFAULT_PART_REFIX = "part";
-
-	/**
-	 * The base {@code Path} that stored all rolling bucket directories.
-	 */
-	private final String basePath;
-
-	/**
-	 * The {@code Bucketer} that is used to determine the path of bucket directories.
-	 */
-	private Bucketer bucketer;
-
-	/**
-	 * We have a template and call duplicate() for each parallel writer in open() to get the actual
-	 * writer that is used for the part files.
-	 */
-	private Writer<T> writerTemplate;
-
-	/**
-	 * The actual writer that we user for writing the part files.
-	 */
-	private Writer<T> writer;
-
-	/**
-	 * Maximum size of part files. If files exceed this we close and create a new one in the same
-	 * bucket directory.
-	 */
-	private long batchSize;
-
-	/**
-	 * If this is true we remove any leftover in-progress/pending files when the sink is opened.
-	 *
-	 * <p>
-	 * This should only be set to false if using the sink without checkpoints, to not remove
-	 * the files already in the directory.
-	 */
-	private boolean cleanupOnOpen = true;
-
-	// These are the actually configured prefixes/suffixes
-	private String inProgressSuffix = DEFAULT_IN_PROGRESS_SUFFIX;
-	private String inProgressPrefix = DEFAULT_IN_PROGRESS_PREFIX;
-
-	private String pendingSuffix = DEFAULT_PENDING_SUFFIX;
-	private String pendingPrefix = DEFAULT_PENDING_PREFIX;
-
-	private String validLengthSuffix = DEFAULT_VALID_SUFFIX;
-	private String validLengthPrefix= DEFAULT_VALID_PREFIX;
-
-	private String partPrefix = DEFAULT_PART_REFIX;
-
-	/**
-	 * The part file that we are currently writing to.
-	 */
-	private transient Path currentPartPath;
-
-	/**
-	 * The bucket directory that we are currently filling.
-	 */
-	private transient Path currentBucketDirectory;
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal fields (not configurable by user)
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * The {@code FSDataOutputStream} for the current part file.
-	 */
-	private transient FSDataOutputStream outStream;
-
-	/**
-	 * Our subtask index, retrieved from the {@code RuntimeContext} in {@link #open}.
-	 */
-	private transient int subtaskIndex;
-
-	/**
-	 * For counting the part files inside a bucket directory. Part files follow the patter
-	 * {@code "{part-prefix}-{subtask}-{count}"}. When creating new part files we increase the counter.
-	 */
-	private transient int partCounter;
-
-	/**
-	 * We use reflection to get the hflush method or use sync as a fallback.
-	 * The idea for this and the code comes from the Flume HDFS Sink.
-	 */
-	private transient Method refHflushOrSync;
-
-	/**
-	 * We use reflection to get the .truncate() method, this is only available starting with
-	 * Hadoop 2.7
-	 */
-	private transient Method refTruncate;
-
-	/**
-	 * The state object that is handled by flink from snapshot/restore. In there we store the
-	 * current part file path, the valid length of the in-progress files and pending part files.
-	 */
-	private transient BucketState bucketState;
-
-	/**
-	 * Creates a new {@code RollingSink} that writes files to the given base directory.
-	 *
-	 * <p>
-	 * This uses a{@link DateTimeBucketer} as bucketer and a {@link StringWriter} has writer.
-	 * The maximum bucket size is set to 384 MB.
-	 *
-	 * @param basePath The directory to which to write the bucket files.
-	 */
-	public RollingSink(String basePath) {
-		this.basePath = basePath;
-		this.bucketer = new DateTimeBucketer();
-		this.batchSize = DEFAULT_BATCH_SIZE;
-		this.writerTemplate = new StringWriter<>();
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
-		if (this.writerTemplate instanceof InputTypeConfigurable) {
-			((InputTypeConfigurable) writerTemplate).setInputType(type, executionConfig);
-		}
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		partCounter = 0;
-
-		this.writer = writerTemplate.duplicate();
-
-		if (bucketState == null) {
-			bucketState = new BucketState();
-		}
-
-		FileSystem fs = new Path(basePath).getFileSystem(new org.apache.hadoop.conf.Configuration());
-		refTruncate = reflectTruncate(fs);
-
-		// delete pending/in-progress files that might be left if we fail while
-		// no checkpoint has yet been done
-		try {
-			if (fs.exists(new Path(basePath)) && cleanupOnOpen) {
-				RemoteIterator<LocatedFileStatus> bucketFiles = fs.listFiles(new Path(basePath), true);
-
-				while (bucketFiles.hasNext()) {
-					LocatedFileStatus file = bucketFiles.next();
-					if (file.getPath().toString().endsWith(pendingSuffix)) {
-						// only delete files that contain our subtask index
-						if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
-							LOG.debug("Deleting leftover pending file {}", file.getPath().toString());
-							fs.delete(file.getPath(), true);
-						}
-					}
-					if (file.getPath().toString().endsWith(inProgressSuffix)) {
-						// only delete files that contain our subtask index
-						if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
-							LOG.debug("Deleting leftover in-progress file {}", file.getPath().toString());
-							fs.delete(file.getPath(), true);
-						}
-					}
-				}
-			}
-		} catch (IOException e) {
-			LOG.error("Error while deleting leftover pending/in-progress files: {}", e);
-			throw new RuntimeException("Error while deleting leftover pending/in-progress files.", e);
-		}
-	}
-
-	@Override
-	public void close() throws Exception {
-//		boolean interrupted = Thread.interrupted();
-		closeCurrentPartFile();
-
-//		if (interrupted) {
-//			Thread.currentThread().interrupt();
-//		}
-	}
-
-	@Override
-	public void invoke(T value) throws Exception {
-
-		if (shouldRoll()) {
-			openNewPartFile();
-		}
-
-		writer.write(value);
-	}
-
-	/**
-	 * Determines whether we should change the bucket file we are writing to.
-	 *
-	 * <p>
-	 * This will roll if no file was created yet, if the file size is larger than the specified size
-	 * or if the {@code Bucketer} determines that we should roll.
-	 */
-	private boolean shouldRoll() throws IOException {
-		boolean shouldRoll = false;
-		if (outStream == null) {
-			shouldRoll = true;
-			LOG.debug("RollingSink {} starting new initial bucket. ", subtaskIndex);
-		}
-		if (bucketer.shouldStartNewBucket(new Path(basePath), currentBucketDirectory)) {
-			shouldRoll = true;
-			LOG.debug("RollingSink {} starting new bucket because {} said we should. ", subtaskIndex, bucketer);
-			// we will retrieve a new bucket base path in openNewPartFile so reset the part counter
-			partCounter = 0;
-		}
-		if (outStream != null) {
-			long writePosition = outStream.getPos();
-			if (outStream != null && writePosition > batchSize) {
-				shouldRoll = true;
-				LOG.debug(
-						"RollingSink {} starting new bucket because file position {} is above batch size {}.",
-						subtaskIndex,
-						writePosition,
-						batchSize);
-			}
-		}
-		return shouldRoll;
-	}
-
-	/**
-	 * Opens a new part file.
-	 *
-	 * <p>
-	 * This closes the old bucket file and retrieves a new bucket path from the {@code Bucketer}.
-	 */
-	private void openNewPartFile() throws Exception {
-		closeCurrentPartFile();
-
-		org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration();
-
-		FileSystem fs = new Path(basePath).getFileSystem(conf);
-
-		Path newBucketDirectory = bucketer.getNextBucketPath(new Path(basePath));
-
-		if (!newBucketDirectory.equals(currentBucketDirectory)) {
-			currentBucketDirectory = newBucketDirectory;
-			try {
-				if (fs.mkdirs(currentBucketDirectory)) {
-					LOG.debug("Created new bucket directory: {}", currentBucketDirectory);
-				}
-			} catch (IOException e) {
-				throw new RuntimeException("Could not create base path for new rolling file.", e);
-			}
-		}
-
-
-		currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
-
-		// This should work since there is only one parallel subtask that tries names with
-		// our subtask id. Otherwise we would run into concurrency issues here.
-		while (fs.exists(currentPartPath) || fs.exists(new Path(currentPartPath.getParent(), pendingPrefix + currentPartPath.getName()).suffix(pendingSuffix))) {
-			partCounter++;
-			currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
-		}
-
-		// increase, so we don't have to check for this name next time
-		partCounter++;
-
-		LOG.debug("Next part path is {}", currentPartPath.toString());
-
-		Path inProgressPath = new Path(currentPartPath.getParent(), inProgressPrefix + currentPartPath.getName()).suffix(inProgressSuffix);
-
-
-
-		outStream = fs.create(inProgressPath, false);
-
-		// We do the reflection here since this is the first time that we have a FSDataOutputStream
-		if (refHflushOrSync == null) {
-			refHflushOrSync = reflectHflushOrSync(outStream);
-		}
-
-		writer.open(outStream);
-	}
-
-	/**
-	 * Closes the current part file.
-	 *
-	 * <p>
-	 * This moves the current in-progress part file to a pending file and adds it to the list
-	 * of pending files in our bucket state.
-	 */
-	private void closeCurrentPartFile() throws Exception {
-		if (writer != null) {
-			writer.close();
-		}
-
-		if (outStream != null) {
-			hflushOrSync(outStream);
-			outStream.close();
-			outStream = null;
-		}
-		if (currentPartPath != null) {
-			Path inProgressPath = new Path(currentPartPath.getParent(), inProgressPrefix + currentPartPath.getName()).suffix(inProgressSuffix);
-			Path pendingPath = new Path(currentPartPath.getParent(), pendingPrefix + currentPartPath.getName()).suffix(pendingSuffix);
-			FileSystem fs = inProgressPath.getFileSystem(new org.apache.hadoop.conf.Configuration());
-			fs.rename(inProgressPath, pendingPath);
-			LOG.debug("Moving in-progress bucket {} to pending file {}",
-					inProgressPath,
-					pendingPath);
-			this.bucketState.pendingFiles.add(currentPartPath.toString());
-		}
-	}
-
-	/**
-	 * If hflush is available in this version of HDFS, then this method calls
-	 * hflush, else it calls sync.
-	 * @param os - The stream to flush/sync
-	 * @throws java.io.IOException
-	 *
-	 * <p>
-	 * Note: This code comes from Flume
-	 */
-	protected void hflushOrSync(FSDataOutputStream os) throws IOException {
-		try {
-			// At this point the refHflushOrSync cannot be null,
-			// since register method would have thrown if it was.
-			this.refHflushOrSync.invoke(os);
-		} catch (InvocationTargetException e) {
-			String msg = "Error while trying to hflushOrSync!";
-			LOG.error(msg + " " + e.getCause());
-			Throwable cause = e.getCause();
-			if(cause != null && cause instanceof IOException) {
-				throw (IOException)cause;
-			}
-			throw new RuntimeException(msg, e);
-		} catch (Exception e) {
-			String msg = "Error while trying to hflushOrSync!";
-			LOG.error(msg + " " + e);
-			throw new RuntimeException(msg, e);
-		}
-	}
-
-	/**
-	 * Gets the hflush call using reflection. Fallback to sync if hflush is not available.
-	 *
-	 * <p>
-	 * Note: This code comes from Flume
-	 */
-	private Method reflectHflushOrSync(FSDataOutputStream os) {
-		Method m = null;
-		if(os != null) {
-			Class<?> fsDataOutputStreamClass = os.getClass();
-			try {
-				m = fsDataOutputStreamClass.getMethod("hflush");
-			} catch (NoSuchMethodException ex) {
-				LOG.debug("HFlush not found. Will use sync() instead");
-				try {
-					m = fsDataOutputStreamClass.getMethod("sync");
-				} catch (Exception ex1) {
-					String msg = "Neither hflush not sync were found. That seems to be " +
-							"a problem!";
-					LOG.error(msg);
-					throw new RuntimeException(msg, ex1);
-				}
-			}
-		}
-		return m;
-	}
-
-	/**
-	 * Gets the truncate() call using reflection.
-	 *
-	 * <p>
-	 * Note: This code comes from Flume
-	 */
-	private Method reflectTruncate(FileSystem fs) {
-		Method m = null;
-		if(fs != null) {
-			Class<?> fsClass = fs.getClass();
-			try {
-				m = fsClass.getMethod("truncate", Path.class, long.class);
-			} catch (NoSuchMethodException ex) {
-				LOG.debug("Truncate not found. Will write a file with suffix '{}' " +
-						" and prefix '{}' to specify how many bytes in a bucket are valid.", validLengthSuffix, validLengthPrefix);
-				return null;
-			}
-
-
-			// verify that truncate actually works
-			FSDataOutputStream outputStream;
-			Path testPath = new Path(UUID.randomUUID().toString());
-			try {
-				outputStream = fs.create(testPath);
-				outputStream.writeUTF("hello");
-				outputStream.close();
-			} catch (IOException e) {
-				LOG.error("Could not create file for checking if truncate works.", e);
-				throw new RuntimeException("Could not create file for checking if truncate works.", e);
-			}
-
-
-			try {
-				m.invoke(fs, testPath, 2);
-			} catch (IllegalAccessException | InvocationTargetException e) {
-				LOG.debug("Truncate is not supported.", e);
-				m = null;
-			}
-
-			try {
-				fs.delete(testPath, false);
-			} catch (IOException e) {
-				LOG.error("Could not delete truncate test file.", e);
-				throw new RuntimeException("Could not delete truncate test file.", e);
-			}
-		}
-		return m;
-	}
-
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		synchronized (bucketState.pendingFilesPerCheckpoint) {
-			Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
-			Set<Long> checkpointsToRemove = Sets.newHashSet();
-			for (Long pastCheckpointId : pastCheckpointIds) {
-				if (pastCheckpointId <= checkpointId) {
-					LOG.debug("Moving pending files to final location for checkpoint {}", pastCheckpointId);
-					// All the pending files are buckets that have been completed but are waiting to be renamed
-					// to their final name
-					for (String filename : bucketState.pendingFilesPerCheckpoint.get(
-							pastCheckpointId)) {
-						Path finalPath = new Path(filename);
-						Path pendingPath = new Path(finalPath.getParent(),
-								pendingPrefix + finalPath.getName()).suffix(pendingSuffix);
-
-						FileSystem fs = pendingPath.getFileSystem(new org.apache.hadoop.conf.Configuration());
-						fs.rename(pendingPath, finalPath);
-						LOG.debug(
-								"Moving pending file {} to final location after complete checkpoint {}.",
-								pendingPath,
-								pastCheckpointId);
-					}
-					checkpointsToRemove.add(pastCheckpointId);
-				}
-			}
-			for (Long toRemove: checkpointsToRemove) {
-				bucketState.pendingFilesPerCheckpoint.remove(toRemove);
-			}
-		}
-	}
-
-	@Override
-	public BucketState snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		if (writer != null) {
-			writer.flush();
-		}
-		if (outStream != null) {
-			hflushOrSync(outStream);
-			bucketState.currentFile = currentPartPath.toString();
-			bucketState.currentFileValidLength = outStream.getPos();
-		}
-		synchronized (bucketState.pendingFilesPerCheckpoint) {
-			bucketState.pendingFilesPerCheckpoint.put(checkpointId, bucketState.pendingFiles);
-		}
-		bucketState.pendingFiles = Lists.newArrayList();
-		return bucketState;
-	}
-
-	@Override
-	public void restoreState(BucketState state) {
-		bucketState = state;
-		// we can clean all the pending files since they where renamed to final files
-		// after this checkpoint was successfull
-		bucketState.pendingFiles.clear();
-		FileSystem fs = null;
-		try {
-			fs = new Path(basePath).getFileSystem(new org.apache.hadoop.conf.Configuration());
-		} catch (IOException e) {
-			LOG.error("Error while creating FileSystem in checkpoint restore.", e);
-			throw new RuntimeException("Error while creating FileSystem in checkpoint restore.", e);
-		}
-		if (bucketState.currentFile != null) {
-			// We were writing to a file when the last checkpoint occured. This file can either
-			// be still in-progress or became a pending file at some point after the checkpoint.
-			// Either way, we have to truncate it back to a valid state (or write a .valid-length)
-			// file that specifies up to which length it is valid and rename it to the final name
-			// before starting a new bucket file.
-			Path partPath = new Path(bucketState.currentFile);
-			try {
-				Path partPendingPath = new Path(partPath.getParent(), pendingPrefix + partPath.getName()).suffix(
-						pendingSuffix);
-				Path partInProgressPath = new Path(partPath.getParent(), inProgressPrefix + partPath.getName()).suffix(inProgressSuffix);
-
-				if (fs.exists(partPendingPath)) {
-					LOG.debug("In-progress file {} has been moved to pending after checkpoint, moving to final location.", partPath);
-					// has been moved to pending in the mean time, rename to final location
-					fs.rename(partPendingPath, partPath);
-				} else if (fs.exists(partInProgressPath)) {
-					LOG.debug("In-progress file {} is still in-progress, moving to final location.", partPath);
-					// it was still in progress, rename to final path
-					fs.rename(partInProgressPath, partPath);
-				} else {
-					LOG.error("In-Progress file {} was neither moved to pending nor is still in progress.", bucketState.currentFile);
-					throw new RuntimeException("In-Progress file " + bucketState.currentFile+ " " +
-							"was neither moved to pending nor is still in progress.");
-				}
-
-				refTruncate = reflectTruncate(fs);
-				// truncate it or write a ".valid-length" file to specify up to which point it is valid
-				if (refTruncate != null) {
-					LOG.debug("Truncating {} to valid length {}", partPath, bucketState.currentFileValidLength);
-					refTruncate.invoke(fs, partPath, bucketState.currentFileValidLength);
-				} else {
-					LOG.debug("Writing valid-length file for {} to specify valid length {}", partPath, bucketState.currentFileValidLength);
-					Path validLengthFilePath = new Path(partPath.getParent(), validLengthPrefix + partPath.getName()).suffix(validLengthSuffix);
-					FSDataOutputStream lengthFileOut = fs.create(validLengthFilePath);
-					lengthFileOut.writeUTF(Long.toString(bucketState.currentFileValidLength));
-					lengthFileOut.close();
-				}
-
-				// invalidate in the state object
-				bucketState.currentFile = null;
-				bucketState.currentFileValidLength = -1;
-			} catch (IOException e) {
-				LOG.error("Error while restoring RollingSink state.", e);
-				throw new RuntimeException("Error while restoring RollingSink state.", e);
-			} catch (InvocationTargetException | IllegalAccessException e) {
-				LOG.error("Cound not invoke truncate.", e);
-				throw new RuntimeException("Could not invoke truncate.", e);
-			}
-		}
-
-		LOG.debug("Clearing pending/in-progress files.");
-
-		// Move files that are confirmed by a checkpoint but did not get moved to final location
-		// because the checkpoint notification did not happen before a failure
-
-		Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
-		LOG.debug("Moving pending files to final location on restore.");
-		for (Long pastCheckpointId : pastCheckpointIds) {
-			// All the pending files are buckets that have been completed but are waiting to be renamed
-			// to their final name
-			for (String filename : bucketState.pendingFilesPerCheckpoint.get(pastCheckpointId)) {
-				Path finalPath = new Path(filename);
-				Path pendingPath = new Path(finalPath.getParent(), pendingPrefix + finalPath.getName()).suffix(pendingSuffix);
-
-				try {
-					if (fs.exists(pendingPath)) {
-						LOG.debug(
-								"Moving pending file {} to final location after complete checkpoint {}.",
-								pendingPath,
-								pastCheckpointId);
-						fs.rename(pendingPath, finalPath);
-					}
-				} catch (IOException e) {
-					LOG.error("Error while renaming pending file {} to final path {}: {}", pendingPath, finalPath, e);
-					throw new RuntimeException("Error while renaming pending file " + pendingPath+ " to final path " + finalPath, e);
-				}
-			}
-		}
-		bucketState.pendingFiles.clear();
-		synchronized (bucketState.pendingFilesPerCheckpoint) {
-			bucketState.pendingFilesPerCheckpoint.clear();
-		}
-
-		// we need to get this here since open() has not yet been called
-		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
-		// delete pending files
-		try {
-
-			RemoteIterator<LocatedFileStatus> bucketFiles = fs.listFiles(new Path(basePath), true);
-
-			while (bucketFiles.hasNext()) {
-				LocatedFileStatus file = bucketFiles.next();
-				if (file.getPath().toString().endsWith(pendingSuffix)) {
-					// only delete files that contain our subtask index
-					if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
-						fs.delete(file.getPath(), true);
-					}
-				}
-				if (file.getPath().toString().endsWith(inProgressSuffix)) {
-					// only delete files that contain our subtask index
-					if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
-						LOG.debug("Deleting in-progress file {}", file.getPath().toString());
-						fs.delete(file.getPath(), true);
-					}
-				}
-			}
-		} catch (IOException e) {
-			LOG.error("Error while deleting old pending files: {}", e);
-			throw new RuntimeException("Error while deleting old pending files.", e);
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Setters for User configuration values
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Sets the maximum bucket size in bytes.
-	 *
-	 * <p>
-	 * When a bucket part file becomes larger than this size a new bucket part file is started and
-	 * the old one is closed. The name of the bucket files depends on the {@link Bucketer}.
-	 *
-	 * @param batchSize The bucket part file size in bytes.
-	 */
-	public RollingSink<T> setBatchSize(long batchSize) {
-		this.batchSize = batchSize;
-		return this;
-	}
-
-	/**
-	 * Sets the {@link Bucketer} to use for determining the bucket files to write to.
-	 *
-	 * @param bucketer The bucketer to use.
-	 */
-	public RollingSink<T> setBucketer(Bucketer bucketer) {
-		this.bucketer = bucketer;
-		return this;
-	}
-
-	/**
-	 * Sets the {@link Writer} to be used for writing the incoming elements to bucket files.
-	 *
-	 * @param writer The {@code Writer} to use.
-	 */
-	public RollingSink<T> setWriter(Writer<T> writer) {
-		this.writerTemplate = writer;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of in-progress part files. The default is {@code "in-progress"}.
-	 */
-	public RollingSink<T> setInProgressSuffix(String inProgressSuffix) {
-		this.inProgressSuffix = inProgressSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of in-progress part files. The default is {@code "_"}.
-	 */
-	public RollingSink<T> setInProgressPrefix(String inProgressPrefix) {
-		this.inProgressPrefix = inProgressPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of pending part files. The default is {@code ".pending"}.
-	 */
-	public RollingSink<T> setPendingSuffix(String pendingSuffix) {
-		this.pendingSuffix = pendingSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of pending part files. The default is {@code "_"}.
-	 */
-	public RollingSink<T> setPendingPrefix(String pendingPrefix) {
-		this.pendingPrefix = pendingPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the suffix of valid-length files. The default is {@code ".valid-length"}.
-	 */
-	public RollingSink<T> setValidLengthSuffix(String validLengthSuffix) {
-		this.validLengthSuffix = validLengthSuffix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of valid-length files. The default is {@code "_"}.
-	 */
-	public RollingSink<T> setValidLengthPrefix(String validLengthPrefix) {
-		this.validLengthPrefix = validLengthPrefix;
-		return this;
-	}
-
-	/**
-	 * Sets the prefix of part files.  The default is {@code "part"}.
-	 */
-	public RollingSink<T> setPartPrefix(String partPrefix) {
-		this.partPrefix = partPrefix;
-		return this;
-	}
-
-	/**
-	 * Disable cleanup of leftover in-progress/pending files when the sink is opened.
-	 *
-	 * <p>
-	 * This should only be disabled if using the sink without checkpoints, to not remove
-	 * the files already in the directory.
-	 */
-	public RollingSink<T> disableCleanupOnOpen() {
-		this.cleanupOnOpen = false;
-		return this;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Internal Classes
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * This is used for keeping track of the current in-progress files and files that we mark
-	 * for moving from pending to final location after we get a checkpoint-complete notification.
-	 */
-	static final class BucketState implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		/**
-		 * The file that was in-progress when the last checkpoint occured.
-		 */
-		String currentFile = null;
-
-		/**
-		 * The valid length of the in-progress file at the time of the last checkpoint.
-		 */
-		long currentFileValidLength = -1;
-
-		/**
-		 * Pending files that accumulated since the last checkpoint.
-		 */
-		List<String> pendingFiles = Lists.newArrayList();
-
-		/**
-		 * When doing a checkpoint we move the pending files since the last checkpoint to this map
-		 * with the id of the checkpoint. When we get the checkpoint-complete notification we move
-		 * pending files of completed checkpoints to their final location.
-		 */
-		final Map<Long, List<String>> pendingFilesPerCheckpoint = Maps.newHashMap();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
deleted file mode 100644
index 928d96e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
+++ /dev/null
@@ -1,160 +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.connectors.fs;
-
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-
-import java.io.IOException;
-
-/**
- * A {@link Writer} that writes the bucket files as Hadoop {@link SequenceFile SequenceFiles}.
- * The input to the {@link RollingSink} must
- * be a {@link org.apache.flink.api.java.tuple.Tuple2} of two Hadopo
- * {@link org.apache.hadoop.io.Writable Writables}.
- *
- * @param <K> The type of the first tuple field.
- * @param <V> The type of the second tuple field.
- */
-public class SequenceFileWriter<K extends Writable, V extends Writable> implements Writer<Tuple2<K, V>>, InputTypeConfigurable {
-	private static final long serialVersionUID = 1L;
-
-	private final String compressionCodecName;
-
-	private SequenceFile.CompressionType compressionType;
-
-	private transient FSDataOutputStream outputStream;
-
-	private transient SequenceFile.Writer writer;
-
-	private Class<K> keyClass;
-
-	private Class<V> valueClass;
-
-	/**
-	 * Creates a new {@code SequenceFileWriter} that writes sequence files without compression.
-	 */
-	public SequenceFileWriter() {
-		this("None", SequenceFile.CompressionType.NONE);
-	}
-
-	/**
-	 * Creates a new {@code SequenceFileWriter} that writes sequence with the given
-	 * compression codec and compression type.
-	 *
-	 * @param compressionCodecName Name of a Hadoop Compression Codec.
-	 * @param compressionType The compression type to use.
-	 */
-	public SequenceFileWriter(String compressionCodecName,
-			SequenceFile.CompressionType compressionType) {
-		this.compressionCodecName = compressionCodecName;
-		this.compressionType = compressionType;
-	}
-
-	@Override
-	public void open(FSDataOutputStream outStream) throws IOException {
-		if (outputStream != null) {
-			throw new IllegalStateException("SequenceFileWriter has already been opened.");
-		}
-		if (keyClass == null) {
-			throw new IllegalStateException("Key Class has not been initialized.");
-		}
-		if (valueClass == null) {
-			throw new IllegalStateException("Value Class has not been initialized.");
-		}
-
-		this.outputStream = outStream;
-
-		CompressionCodec codec = null;
-
-		if (!compressionCodecName.equals("None")) {
-			CompressionCodecFactory codecFactory = new CompressionCodecFactory(new Configuration());
-			codec = codecFactory.getCodecByName(compressionCodecName);
-			if (codec == null) {
-				throw new RuntimeException("Codec " + compressionCodecName + " not found.");
-			}
-		}
-
-		// the non-deprecated constructor syntax is only available in recent hadoop versions...
-		writer = SequenceFile.createWriter(new Configuration(),
-				outStream,
-				keyClass,
-				valueClass,
-				compressionType,
-				codec);
-	}
-
-	@Override
-	public void flush() throws IOException {
-	}
-
-	@Override
-	public void close() throws IOException {
-		if (writer != null) {
-			writer.close();
-		}
-		writer = null;
-		outputStream = null;
-	}
-
-	@Override
-	public void write(Tuple2<K, V> element) throws IOException {
-		if (outputStream == null) {
-			throw new IllegalStateException("SequenceFileWriter has not been opened.");
-		}
-		writer.append(element.f0, element.f1);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
-		if (!type.isTupleType()) {
-			throw new IllegalArgumentException("Input TypeInformation is not a tuple type.");
-		}
-
-		TupleTypeInfoBase<?> tupleType = (TupleTypeInfoBase<?>) type;
-
-		if (tupleType.getArity() != 2) {
-			throw new IllegalArgumentException("Input TypeInformation must be a Tuple2 type.");
-		}
-
-		TypeInformation<K> keyType = tupleType.getTypeAt(0);
-		TypeInformation<V> valueType = tupleType.getTypeAt(1);
-
-		this.keyClass = keyType.getTypeClass();
-		this.valueClass = valueType.getTypeClass();
-	}
-
-	@Override
-	public Writer<Tuple2<K, V>> duplicate() {
-		SequenceFileWriter<K, V> result = new SequenceFileWriter<>(compressionCodecName, compressionType);
-		result.keyClass = keyClass;
-		result.valueClass = valueClass;
-		return result;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java
deleted file mode 100644
index ad0ab46..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java
+++ /dev/null
@@ -1,103 +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.connectors.fs;
-
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.nio.charset.IllegalCharsetNameException;
-import java.nio.charset.UnsupportedCharsetException;
-
-/**
- * A {@link Writer} that uses {@code toString()} on the input elements and writes them to
- * the output bucket file separated by newline.
- *
- * @param <T> The type of the elements that are being written by the sink.
- */
-public class StringWriter<T> implements Writer<T> {
-	private static final long serialVersionUID = 1L;
-
-	private transient FSDataOutputStream outputStream;
-
-	private String charsetName;
-
-	private transient Charset charset;
-
-	/**
-	 * Creates a new {@code StringWriter} that uses {@code "UTF-8"} charset to convert
-	 * strings to bytes.
-	 */
-	public StringWriter() {
-		this("UTF-8");
-	}
-
-	/**
-	 * Creates a new {@code StringWriter} that uses the given charset to convert
-	 * strings to bytes.
-	 *
-	 * @param charsetName Name of the charset to be used, must be valid input for {@code Charset.forName(charsetName)}
-	 */
-	public StringWriter(String charsetName) {
-		this.charsetName = charsetName;
-	}
-
-	@Override
-	public void open(FSDataOutputStream outStream) throws IOException {
-		if (outputStream != null) {
-			throw new IllegalStateException("StringWriter has already been opened.");
-		}
-		this.outputStream = outStream;
-
-		try {
-			this.charset = Charset.forName(charsetName);
-		}
-		catch (IllegalCharsetNameException e) {
-			throw new IOException("The charset " + charsetName + " is not valid.", e);
-		}
-		catch (UnsupportedCharsetException e) {
-			throw new IOException("The charset " + charsetName + " is not supported.", e);
-		}
-	}
-
-	@Override
-	public void flush() throws IOException {
-
-	}
-
-	@Override
-	public void close() throws IOException {
-		outputStream = null;
-	}
-
-	@Override
-	public void write(T element) throws IOException {
-		if (outputStream == null) {
-			throw new IllegalStateException("StringWriter has not been opened.");
-		}
-		outputStream.write(element.toString().getBytes(charset));
-		outputStream.write('\n');
-
-	}
-
-	@Override
-	public Writer<T> duplicate() {
-		return new StringWriter<>();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java
deleted file mode 100644
index 2bab8cf..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java
+++ /dev/null
@@ -1,28 +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.connectors.fs;
-
-/**
- * A {@link Clock} that uses {@code System.currentTimeMillis()} to determine the system time.
- */
-public class SystemClock implements Clock {
-	@Override
-	public long currentTimeMillis() {
-		return System.currentTimeMillis();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
deleted file mode 100644
index 98cad32..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.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.connectors.fs;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * An implementation of {@code Writer} is used in conjunction with a
- * {@link RollingSink} to perform the actual
- * writing to the bucket files.
- *
- * @param <T> The type of the elements that are being written by the sink.
- */
-public interface Writer<T> extends Serializable {
-
-	/**
-	 * Initializes the {@code Writer} for a newly opened bucket file.
-	 * Any internal per-bucket initialization should be performed here.
-	 *
-	 * @param outStream The {@link org.apache.hadoop.fs.FSDataOutputStream} for the newly opened file.
-	 */
-	void open(FSDataOutputStream outStream) throws IOException;
-
-	/**
-	 * Flushes out any internally held data.
-	 */
-	void flush()throws IOException ;
-
-	/**
-	 * Closes the {@code Writer}. This must not close the {@code FSDataOutputStream} that
-	 * was handed in in the {@link #open} method. Only internally held state should be
-	 * closed.
-	 */
-	void close() throws IOException ;
-
-	/**
-	 * Writes one element to the bucket file.
-	 */
-	void write(T element)throws IOException;
-
-	/**
-	 * Duplicates the {@code Writer}. This is used to get one {@code Writer} for each
-	 * parallel instance of the sink.
-	 */
-	Writer<T> duplicate();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties
deleted file mode 100644
index fe60d94..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-log4j.rootLogger=OFF, testlogger
-
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
-
-# suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger


[20/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java
deleted file mode 100644
index 232485d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java
+++ /dev/null
@@ -1,281 +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.tasks;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * These tests verify that the RichFunction methods are called (in correct order). And that
- * checkpointing/element emission don't occur concurrently.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ResultPartitionWriter.class})
-public class SourceStreamTaskTest {
-
-
-	/**
-	 * This test verifies that open() and close() are correctly called by the StreamTask.
-	 */
-	@Test
-	public void testOpenClose() throws Exception {
-		final SourceStreamTask<String> sourceTask = new SourceStreamTask<String>();
-		final StreamTaskTestHarness<String> testHarness = new StreamTaskTestHarness<String>(sourceTask, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		StreamSource<String> sourceOperator = new StreamSource<String>(new OpenCloseTestSource());
-		streamConfig.setStreamOperator(sourceOperator);
-
-		testHarness.invoke();
-		testHarness.waitForTaskCompletion();
-
-		Assert.assertTrue("RichFunction methods where not called.", OpenCloseTestSource.closeCalled);
-
-		List<String> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput());
-		Assert.assertEquals(10, resultElements.size());
-	}
-
-	/**
-	 * This test ensures that the SourceStreamTask properly serializes checkpointing
-	 * and element emission. This also verifies that there are no concurrent invocations
-	 * of the checkpoint method on the source operator.
-	 *
-	 * The source emits elements and performs checkpoints. We have several checkpointer threads
-	 * that fire checkpoint requests at the source task.
-	 *
-	 * If element emission and checkpointing are not in series the count of elements at the
-	 * beginning of a checkpoint and at the end of a checkpoint are not the same because the
-	 * source kept emitting elements while the checkpoint was ongoing.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testCheckpointing() throws Exception {
-		final int NUM_ELEMENTS = 100;
-		final int NUM_CHECKPOINTS = 100;
-		final int NUM_CHECKPOINTERS = 1;
-		final int CHECKPOINT_INTERVAL = 5; // in ms
-		final int SOURCE_CHECKPOINT_DELAY = 1000; // how many random values we sum up in storeCheckpoint
-		final int SOURCE_READ_DELAY = 1; // in ms
-
-		ExecutorService executor = Executors.newFixedThreadPool(10);
-		try {
-			final TupleTypeInfo<Tuple2<Long, Integer>> typeInfo = new TupleTypeInfo<Tuple2<Long, Integer>>(BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO);
-			final SourceStreamTask<Tuple2<Long, Integer>> sourceTask = new SourceStreamTask<Tuple2<Long, Integer>>();
-			final StreamTaskTestHarness<Tuple2<Long, Integer>> testHarness = new StreamTaskTestHarness<Tuple2<Long, Integer>>(sourceTask, typeInfo);
-	
-			StreamConfig streamConfig = testHarness.getStreamConfig();
-			StreamSource<Tuple2<Long, Integer>> sourceOperator = new StreamSource<Tuple2<Long, Integer>>(new MockSource(NUM_ELEMENTS, SOURCE_CHECKPOINT_DELAY, SOURCE_READ_DELAY));
-			streamConfig.setStreamOperator(sourceOperator);
-			
-			// prepare the 
-			
-			Future<Boolean>[] checkpointerResults = new Future[NUM_CHECKPOINTERS];
-	
-			// invoke this first, so the tasks are actually running when the checkpoints are scheduled
-			testHarness.invoke();
-			
-			for (int i = 0; i < NUM_CHECKPOINTERS; i++) {
-				checkpointerResults[i] = executor.submit(new Checkpointer(NUM_CHECKPOINTS, CHECKPOINT_INTERVAL, sourceTask));
-			}
-			
-			testHarness.waitForTaskCompletion();
-	
-			// Get the result from the checkpointers, if these threw an exception it
-			// will be rethrown here
-			for (int i = 0; i < NUM_CHECKPOINTERS; i++) {
-				if (!checkpointerResults[i].isDone()) {
-					checkpointerResults[i].cancel(true);
-				}
-				if (!checkpointerResults[i].isCancelled()) {
-					checkpointerResults[i].get();
-				}
-			}
-	
-			List<Tuple2<Long, Integer>> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput());
-			Assert.assertEquals(NUM_ELEMENTS, resultElements.size());
-		}
-		finally {
-			executor.shutdown();
-		}
-	}
-
-	private static class MockSource implements SourceFunction<Tuple2<Long, Integer>>, Checkpointed<Serializable> {
-		private static final long serialVersionUID = 1;
-
-		private int maxElements;
-		private int checkpointDelay;
-		private int readDelay;
-
-		private volatile int count;
-		private volatile long lastCheckpointId = -1;
-
-		private Semaphore semaphore;
-
-		private volatile boolean isRunning = true;
-
-		public MockSource(int maxElements, int checkpointDelay, int readDelay) {
-			this.maxElements = maxElements;
-			this.checkpointDelay = checkpointDelay;
-			this.readDelay = readDelay;
-			this.count = 0;
-			semaphore = new Semaphore(1);
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Long, Integer>> ctx) {
-			final Object lockObject = ctx.getCheckpointLock();
-			while (isRunning && count < maxElements) {
-				// simulate some work
-				try {
-					Thread.sleep(readDelay);
-				} catch (InterruptedException e) {
-					e.printStackTrace();
-				}
-
-				synchronized (lockObject) {
-					ctx.collect(new Tuple2<Long, Integer>(lastCheckpointId, count));
-					count++;
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		@Override
-		public Serializable snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-			if (!semaphore.tryAcquire()) {
-				Assert.fail("Concurrent invocation of snapshotState.");
-			}
-			int startCount = count;
-			lastCheckpointId = checkpointId;
-
-			long sum = 0;
-			for (int i = 0; i < checkpointDelay; i++) {
-				sum += new Random().nextLong();
-			}
-
-			if (startCount != count) {
-				semaphore.release();
-				// This means that next() was invoked while the snapshot was ongoing
-				Assert.fail("Count is different at start end end of snapshot.");
-			}
-			semaphore.release();
-			return sum;
-		}
-
-		@Override
-		public void restoreState(Serializable state) {
-
-		}
-	}
-
-	/**
-	 * This calls triggerInterrupt on the given task with the given interval.
-	 */
-	private static class Checkpointer implements Callable<Boolean> {
-		private final int numCheckpoints;
-		private final int checkpointInterval;
-		private final AtomicLong checkpointId;
-		private final StreamTask<Tuple2<Long, Integer>, ?> sourceTask;
-
-		public Checkpointer(int numCheckpoints, int checkpointInterval, StreamTask<Tuple2<Long, Integer>, ?> task) {
-			this.numCheckpoints = numCheckpoints;
-			checkpointId = new AtomicLong(0);
-			sourceTask = task;
-			this.checkpointInterval = checkpointInterval;
-		}
-
-		@Override
-		public Boolean call() throws Exception {
-			for (int i = 0; i < numCheckpoints; i++) {
-				long currentCheckpointId = checkpointId.getAndIncrement();
-				sourceTask.triggerCheckpoint(currentCheckpointId, 0L);
-				Thread.sleep(checkpointInterval);
-			}
-			return true;
-		}
-	}
-
-	public static class OpenCloseTestSource extends RichSourceFunction<String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public void run(SourceContext<String> ctx) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			for (int i = 0; i < 10; i++) {
-				ctx.collect("Hello" + i);
-			}
-		}
-
-		@Override
-		public void cancel() {}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
deleted file mode 100644
index 090f7cb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
+++ /dev/null
@@ -1,304 +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.tasks;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.UnmodifiableConfiguration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer;
-import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.Collections;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.concurrent.Future;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class StreamMockEnvironment implements Environment {
-
-	private final MemoryManager memManager;
-
-	private final IOManager ioManager;
-
-	private final InputSplitProvider inputSplitProvider;
-
-	private final Configuration jobConfiguration;
-
-	private final Configuration taskConfiguration;
-
-	private final List<InputGate> inputs;
-
-	private final List<ResultPartitionWriter> outputs;
-
-	private final JobID jobID = new JobID();
-
-	private final BroadcastVariableManager bcVarManager = new BroadcastVariableManager();
-
-	private final AccumulatorRegistry accumulatorRegistry;
-
-	private final int bufferSize;
-
-	public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, long memorySize,
-									MockInputSplitProvider inputSplitProvider, int bufferSize) {
-		this.jobConfiguration = jobConfig;
-		this.taskConfiguration = taskConfig;
-		this.inputs = new LinkedList<InputGate>();
-		this.outputs = new LinkedList<ResultPartitionWriter>();
-
-		this.memManager = new MemoryManager(memorySize, 1);
-		this.ioManager = new IOManagerAsync();
-		this.inputSplitProvider = inputSplitProvider;
-		this.bufferSize = bufferSize;
-
-		this.accumulatorRegistry = new AccumulatorRegistry(jobID, getExecutionId());
-	}
-
-	public void addInputGate(InputGate gate) {
-		inputs.add(gate);
-	}
-
-	public <T> void addOutput(final Queue<Object> outputList, final TypeSerializer<T> serializer) {
-		try {
-			// The record-oriented writers wrap the buffer writer. We mock it
-			// to collect the returned buffers and deserialize the content to
-			// the output list
-			BufferProvider mockBufferProvider = mock(BufferProvider.class);
-			when(mockBufferProvider.requestBufferBlocking()).thenAnswer(new Answer<Buffer>() {
-
-				@Override
-				public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
-					return new Buffer(
-							MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
-							mock(BufferRecycler.class));
-				}
-			});
-
-			ResultPartitionWriter mockWriter = mock(ResultPartitionWriter.class);
-			when(mockWriter.getNumberOfOutputChannels()).thenReturn(1);
-			when(mockWriter.getBufferProvider()).thenReturn(mockBufferProvider);
-
-			final RecordDeserializer<DeserializationDelegate<T>> recordDeserializer = new AdaptiveSpanningRecordDeserializer<DeserializationDelegate<T>>();
-			final NonReusingDeserializationDelegate<T> delegate = new NonReusingDeserializationDelegate<T>(serializer);
-
-			// Add records from the buffer to the output list
-			doAnswer(new Answer<Void>() {
-
-				@Override
-				public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
-					Buffer buffer = (Buffer) invocationOnMock.getArguments()[0];
-
-					recordDeserializer.setNextBuffer(buffer);
-
-					while (recordDeserializer.hasUnfinishedData()) {
-						RecordDeserializer.DeserializationResult result = recordDeserializer.getNextRecord(delegate);
-
-						if (result.isFullRecord()) {
-							outputList.add(delegate.getInstance());
-						}
-
-						if (result == RecordDeserializer.DeserializationResult.LAST_RECORD_FROM_BUFFER
-								|| result == RecordDeserializer.DeserializationResult.PARTIAL_RECORD) {
-							break;
-						}
-					}
-
-					return null;
-				}
-			}).when(mockWriter).writeBuffer(any(Buffer.class), anyInt());
-
-			// Add events to the output list
-			doAnswer(new Answer<Void>() {
-
-				@Override
-				public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
-					AbstractEvent event = (AbstractEvent) invocationOnMock.getArguments()[0];
-
-					outputList.add(event);
-					return null;
-				}
-			}).when(mockWriter).writeEvent(any(AbstractEvent.class), anyInt());
-
-			doAnswer(new Answer<Void>() {
-
-				@Override
-				public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
-					AbstractEvent event = (AbstractEvent) invocationOnMock.getArguments()[0];
-
-					outputList.add(event);
-					return null;
-				}
-			}).when(mockWriter).writeEventToAllChannels(any(AbstractEvent.class));
-
-			outputs.add(mockWriter);
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail(t.getMessage());
-		}
-	}
-
-	@Override
-	public Configuration getTaskConfiguration() {
-		return this.taskConfiguration;
-	}
-
-	@Override
-	public MemoryManager getMemoryManager() {
-		return this.memManager;
-	}
-
-	@Override
-	public IOManager getIOManager() {
-		return this.ioManager;
-	}
-
-	@Override
-	public JobID getJobID() {
-		return this.jobID;
-	}
-
-	@Override
-	public Configuration getJobConfiguration() {
-		return this.jobConfiguration;
-	}
-
-	@Override
-	public int getNumberOfSubtasks() {
-		return 1;
-	}
-
-	@Override
-	public int getIndexInSubtaskGroup() {
-		return 0;
-	}
-
-	@Override
-	public InputSplitProvider getInputSplitProvider() {
-		return this.inputSplitProvider;
-	}
-
-	@Override
-	public String getTaskName() {
-		return "";
-	}
-
-	@Override
-	public String getTaskNameWithSubtasks() {
-		return "";
-	}
-
-	@Override
-	public ClassLoader getUserClassLoader() {
-		return getClass().getClassLoader();
-	}
-
-	@Override
-	public Map<String, Future<Path>> getDistributedCacheEntries() {
-		return Collections.emptyMap();
-	}
-
-	@Override
-	public ResultPartitionWriter getWriter(int index) {
-		return outputs.get(index);
-	}
-
-	@Override
-	public ResultPartitionWriter[] getAllWriters() {
-		return outputs.toArray(new ResultPartitionWriter[outputs.size()]);
-	}
-
-	@Override
-	public InputGate getInputGate(int index) {
-		return inputs.get(index);
-	}
-
-	@Override
-	public InputGate[] getAllInputGates() {
-		InputGate[] gates = new InputGate[inputs.size()];
-		inputs.toArray(gates);
-		return gates;
-	}
-
-	@Override
-	public JobVertexID getJobVertexId() {
-		return new JobVertexID(new byte[16]);
-	}
-
-	@Override
-	public ExecutionAttemptID getExecutionId() {
-		return new ExecutionAttemptID(0L, 0L);
-	}
-
-	@Override
-	public BroadcastVariableManager getBroadcastVariableManager() {
-		return this.bcVarManager;
-	}
-
-	@Override
-	public AccumulatorRegistry getAccumulatorRegistry() {
-		return accumulatorRegistry;
-	}
-
-	@Override
-	public void acknowledgeCheckpoint(long checkpointId) {
-	}
-
-	@Override
-	public void acknowledgeCheckpoint(long checkpointId, StateHandle<?> state) {
-	}
-
-	@Override
-	public TaskManagerRuntimeInfo getTaskManagerInfo() {
-		return new TaskManagerRuntimeInfo("localhost", new UnmodifiableConfiguration(new Configuration()));
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
deleted file mode 100644
index 6c48668..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ /dev/null
@@ -1,321 +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.tasks;
-
-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.configuration.Configuration;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.streaming.api.collector.selector.BroadcastOutputSelectorWrapper;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.graph.StreamNode;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Assert;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-
-/**
- * Test harness for testing a {@link StreamTask}.
- *
- * <p>
- * This mock Invokable provides the task with a basic runtime context and allows pushing elements
- * and watermarks into the task. {@link #getOutput()} can be used to get the emitted elements
- * and events. You are free to modify the retrieved list.
- *
- * <p>
- * After setting up everything the Task can be invoked using {@link #invoke()}. This will start
- * a new Thread to execute the Task. Use {@link #waitForTaskCompletion()} to wait for the Task
- * thread to finish.
- *
- * <p>
- * When using this you need to add the following line to your test class to setup Powermock:
- * {@code @PrepareForTest({ResultPartitionWriter.class})}
- */
-public class StreamTaskTestHarness<OUT> {
-
-	private static final int DEFAULT_MEMORY_MANAGER_SIZE = 1024 * 1024;
-
-	private static final int DEFAULT_NETWORK_BUFFER_SIZE = 1024;
-
-	protected long memorySize = 0;
-	protected int bufferSize = 0;
-
-	protected StreamMockEnvironment mockEnv;
-	protected ExecutionConfig executionConfig;
-	private Configuration jobConfig;
-	private Configuration taskConfig;
-	protected StreamConfig streamConfig;
-
-	private AbstractInvokable task;
-
-	private TypeSerializer<OUT> outputSerializer;
-	private TypeSerializer<StreamElement> outputStreamRecordSerializer;
-
-	private ConcurrentLinkedQueue<Object> outputList;
-
-	protected TaskThread taskThread;
-
-	// These don't get initialized, the one-input/two-input specific test harnesses
-	// must initialize these if they want to simulate input. We have them here so that all the
-	// input related methods only need to be implemented once, in generic form
-	protected int numInputGates;
-	protected int numInputChannelsPerGate;
-	@SuppressWarnings("rawtypes")
-	protected StreamTestSingleInputGate[] inputGates;
-
-	public StreamTaskTestHarness(AbstractInvokable task, TypeInformation<OUT> outputType) {
-		this.task = task;
-		this.memorySize = DEFAULT_MEMORY_MANAGER_SIZE;
-		this.bufferSize = DEFAULT_NETWORK_BUFFER_SIZE;
-
-		this.jobConfig = new Configuration();
-		this.taskConfig = new Configuration();
-		this.executionConfig = new ExecutionConfig();
-		executionConfig.enableTimestamps();
-		try {
-			InstantiationUtil.writeObjectToConfig(executionConfig, this.jobConfig, ExecutionConfig.CONFIG_KEY);
-		} catch (IOException e) {
-			throw new RuntimeException(e);
-		}
-
-		streamConfig = new StreamConfig(taskConfig);
-		streamConfig.setChainStart();
-		streamConfig.setBufferTimeout(0);
-
-		outputSerializer = outputType.createSerializer(executionConfig);
-		outputStreamRecordSerializer = new MultiplexingStreamRecordSerializer<OUT>(outputSerializer);
-	}
-
-	/**
-	 * This must be overwritten for OneInputStreamTask or TwoInputStreamTask test harnesses.
-	 */
-	protected void initializeInputs() throws IOException, InterruptedException {}
-
-	@SuppressWarnings("unchecked")
-	private void initializeOutput() {
-		outputList = new ConcurrentLinkedQueue<Object>();
-
-		mockEnv.addOutput(outputList, outputStreamRecordSerializer);
-
-		streamConfig.setOutputSelectorWrapper(new BroadcastOutputSelectorWrapper<Object>());
-		streamConfig.setNumberOfOutputs(1);
-
-		StreamOperator<OUT> dummyOperator = new AbstractStreamOperator<OUT>() {
-			private static final long serialVersionUID = 1L;
-		};
-
-		List<StreamEdge> outEdgesInOrder = new LinkedList<StreamEdge>();
-		StreamNode sourceVertexDummy = new StreamNode(null, 0, dummyOperator, "source dummy", new LinkedList<OutputSelector<?>>(), SourceStreamTask.class);
-		StreamNode targetVertexDummy = new StreamNode(null, 1, dummyOperator, "target dummy", new LinkedList<OutputSelector<?>>(), SourceStreamTask.class);
-
-		outEdgesInOrder.add(new StreamEdge(sourceVertexDummy, targetVertexDummy, 0, new LinkedList<String>(), new BroadcastPartitioner<Object>()));
-		streamConfig.setOutEdgesInOrder(outEdgesInOrder);
-		streamConfig.setNonChainedOutputs(outEdgesInOrder);
-		streamConfig.setTypeSerializerOut(outputSerializer);
-		streamConfig.setVertexID(0);
-
-	}
-
-	/**
-	 * Invoke the Task. This resets the output of any previous invocation. This will start a new
-	 * Thread to execute the Task in. Use {@link #waitForTaskCompletion()} to wait for the
-	 * Task thread to finish running.
-	 */
-	public void invoke() throws Exception {
-		mockEnv = new StreamMockEnvironment(jobConfig, taskConfig, memorySize, new MockInputSplitProvider(), bufferSize);
-		task.setEnvironment(mockEnv);
-
-		initializeInputs();
-		initializeOutput();
-
-		task.registerInputOutput();
-
-		taskThread = new TaskThread(task);
-		taskThread.start();
-	}
-
-	public void waitForTaskCompletion() throws Exception {
-		if (taskThread == null) {
-			throw new IllegalStateException("Task thread was not started.");
-		}
-
-		taskThread.join();
-		if (taskThread.getError() != null) {
-			throw new Exception("error in task", taskThread.getError());
-		}
-	}
-
-	/**
-	 * Get all the output from the task. This contains StreamRecords and Events interleaved. Use
-	 * {@link org.apache.flink.streaming.util.TestHarnessUtil#getRawElementsFromOutput(java.util.Queue)}}
-	 * to extract only the StreamRecords.
-	 */
-	public ConcurrentLinkedQueue<Object> getOutput() {
-		return outputList;
-	}
-
-	public StreamConfig getStreamConfig() {
-		return streamConfig;
-	}
-
-	private void shutdownIOManager() throws Exception {
-		this.mockEnv.getIOManager().shutdown();
-		Assert.assertTrue("IO Manager has not properly shut down.", this.mockEnv.getIOManager().isProperlyShutDown());
-	}
-
-	private void shutdownMemoryManager() throws Exception {
-		if (this.memorySize > 0) {
-			MemoryManager memMan = this.mockEnv.getMemoryManager();
-			if (memMan != null) {
-				Assert.assertTrue("Memory Manager managed memory was not completely freed.", memMan.verifyEmpty());
-				memMan.shutdown();
-			}
-		}
-	}
-
-	/**
-	 * Sends the element to input gate 0 on channel 0.
-	 */
-	@SuppressWarnings("unchecked")
-	public void processElement(Object element) {
-		inputGates[0].sendElement(element, 0);
-	}
-
-	/**
-	 * Sends the element to the specified channel on the specified input gate.
-	 */
-	@SuppressWarnings("unchecked")
-	public void processElement(Object element, int inputGate, int channel) {
-		inputGates[inputGate].sendElement(element, channel);
-	}
-
-	/**
-	 * Sends the event to input gate 0 on channel 0.
-	 */
-	public void processEvent(AbstractEvent event) {
-		inputGates[0].sendEvent(event, 0);
-	}
-
-	/**
-	 * Sends the event to the specified channel on the specified input gate.
-	 */
-	public void processEvent(AbstractEvent event, int inputGate, int channel) {
-		inputGates[inputGate].sendEvent(event, channel);
-	}
-
-	/**
-	 * This only returns after all input queues are empty.
-	 */
-	public void waitForInputProcessing() {
-
-
-		// first wait for all input queues to be empty
-		try {
-			Thread.sleep(1);
-		} catch (InterruptedException ignored) {}
-		
-		while (true) {
-			boolean allEmpty = true;
-			for (int i = 0; i < numInputGates; i++) {
-				if (!inputGates[i].allQueuesEmpty()) {
-					allEmpty = false;
-				}
-			}
-			try {
-				Thread.sleep(10);
-			} catch (InterruptedException ignored) {}
-			
-			if (allEmpty) {
-				break;
-			}
-		}
-
-		// then wait for the Task Thread to be in a blocked state
-		// Check whether the state is blocked, this should be the case if it cannot
-		// read more input, i.e. all currently available input has been processed.
-		while (true) {
-			Thread.State state = taskThread.getState();
-			if (state == Thread.State.BLOCKED || state == Thread.State.TERMINATED ||
-					state == Thread.State.WAITING || state == Thread.State.TIMED_WAITING) {
-				break;
-			}
-
-			try {
-				Thread.sleep(1);
-			} catch (InterruptedException ignored) {}
-		}
-	}
-
-	/**
-	 * Notifies all input channels on all input gates that no more input will arrive. This
-	 * will usually make the Task exit from his internal loop.
-	 */
-	public void endInput() {
-		for (int i = 0; i < numInputGates; i++) {
-			inputGates[i].endInput();
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private class TaskThread extends Thread {
-		
-		private final AbstractInvokable task;
-		
-		private volatile Throwable error;
-
-
-		TaskThread(AbstractInvokable task) {
-			super("Task Thread");
-			this.task = task;
-		}
-
-		@Override
-		public void run() {
-			try {
-				task.invoke();
-				shutdownIOManager();
-				shutdownMemoryManager();
-			}
-			catch (Throwable t) {
-				this.error = t;
-			}
-		}
-
-		public Throwable getError() {
-			return error;
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java
deleted file mode 100644
index cdc2c53..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java
+++ /dev/null
@@ -1,313 +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.tasks;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.concurrent.Semaphore;
-
-/**
- * Tests for the timer service of {@code StreamTask}.
- *
- * <p>
- * These tests ensure that exceptions are properly forwarded from the timer thread to
- * the task thread and that operator methods are not invoked concurrently.
- */
-public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
-
-	/**
-	 * Note: this test fails if we don't have the synchronized block in
-	 * {@link org.apache.flink.streaming.runtime.tasks.SourceStreamTask.SourceOutput}
-	 *
-	 * <p>
-	 * This test never finishes if exceptions from the timer thread are not forwarded. Thus
-	 * a success here means that the exception forwarding works.
-	 */
-	@Test
-	public void testOperatorChainedToSource() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataStream<String> source = env.addSource(new InfiniteTestSource());
-
-		source.transform("Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TimerOperator(ChainingStrategy.ALWAYS));
-
-		boolean testSuccess = false;
-		try {
-			env.execute("Timer test");
-		} catch (JobExecutionException e) {
-			if (e.getCause() instanceof TimerException) {
-				TimerException te = (TimerException) e.getCause();
-				if (te.getCause() instanceof RuntimeException) {
-					RuntimeException re = (RuntimeException) te.getCause();
-					if (re.getMessage().equals("TEST SUCCESS")) {
-						testSuccess = true;
-					} else {
-						throw e;
-					}
-				} else {
-					throw e;
-				}
-			} else {
-				throw e;
-			}
-		}
-		Assert.assertTrue(testSuccess);
-	}
-
-	/**
-	 * Note: this test fails if we don't have the synchronized block in
-	 * {@link org.apache.flink.streaming.runtime.tasks.SourceStreamTask.SourceOutput}
-	 */
-	@Test
-	public void testOneInputOperatorWithoutChaining() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataStream<String> source = env.addSource(new InfiniteTestSource());
-
-		source.transform("Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TimerOperator(ChainingStrategy.NEVER));
-
-		boolean testSuccess = false;
-		try {
-			env.execute("Timer test");
-		} catch (JobExecutionException e) {
-			if (e.getCause() instanceof TimerException) {
-				TimerException te = (TimerException) e.getCause();
-				if (te.getCause() instanceof RuntimeException) {
-					RuntimeException re = (RuntimeException) te.getCause();
-					if (re.getMessage().equals("TEST SUCCESS")) {
-						testSuccess = true;
-					} else {
-						throw e;
-					}
-				} else {
-					throw e;
-				}
-			} else {
-				throw e;
-			}
-		}
-		Assert.assertTrue(testSuccess);
-	}
-
-	/**
-	 * Note: this test fails if we don't have the synchronized block in
-	 * {@link org.apache.flink.streaming.runtime.tasks.SourceStreamTask.SourceOutput}
-	 */
-	@Test
-	public void testTwoInputOperatorWithoutChaining() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataStream<String> source = env.addSource(new InfiniteTestSource());
-
-		source.connect(source).transform(
-				"Custom Operator",
-				BasicTypeInfo.STRING_TYPE_INFO,
-				new TwoInputTimerOperator(ChainingStrategy.NEVER));
-
-		boolean testSuccess = false;
-		try {
-			env.execute("Timer test");
-		} catch (JobExecutionException e) {
-			if (e.getCause() instanceof TimerException) {
-				TimerException te = (TimerException) e.getCause();
-				if (te.getCause() instanceof RuntimeException) {
-					RuntimeException re = (RuntimeException) te.getCause();
-					if (re.getMessage().equals("TEST SUCCESS")) {
-						testSuccess = true;
-					} else {
-						throw e;
-					}
-				} else {
-					throw e;
-				}
-			} else {
-				throw e;
-			}
-		}
-		Assert.assertTrue(testSuccess);
-	}
-
-	public static class TimerOperator extends AbstractStreamOperator<String> implements OneInputStreamOperator<String, String>, Triggerable {
-		private static final long serialVersionUID = 1L;
-
-		int numTimers = 0;
-		int numElements = 0;
-
-		private boolean first = true;
-
-		private Semaphore semaphore = new Semaphore(1);
-
-		public TimerOperator(ChainingStrategy chainingStrategy) {
-			setChainingStrategy(chainingStrategy);
-		}
-
-		@Override
-		public void processElement(StreamRecord<String> element) throws Exception {
-			if (!semaphore.tryAcquire()) {
-				Assert.fail("Concurrent invocation of operator functions.");
-			}
-
-			if (first) {
-				registerTimer(System.currentTimeMillis() + 100, this);
-				first = false;
-			}
-			numElements++;
-
-			semaphore.release();
-		}
-
-		@Override
-		public void trigger(long time) throws Exception {
-			if (!semaphore.tryAcquire()) {
-				Assert.fail("Concurrent invocation of operator functions.");
-			}
-
-			try {
-				numTimers++;
-				throwIfDone();
-				registerTimer(System.currentTimeMillis() + 1, this);
-			} finally {
-				semaphore.release();
-			}
-		}
-
-		private void throwIfDone() {
-			if (numTimers > 1000 && numElements > 10_000) {
-				throw new RuntimeException("TEST SUCCESS");
-			}
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-			//ignore
-		}
-	}
-
-	public static class TwoInputTimerOperator extends AbstractStreamOperator<String> implements TwoInputStreamOperator<String, String, String>, Triggerable {
-		private static final long serialVersionUID = 1L;
-
-		int numTimers = 0;
-		int numElements = 0;
-
-		private boolean first = true;
-
-		private Semaphore semaphore = new Semaphore(1);
-
-		public TwoInputTimerOperator(ChainingStrategy chainingStrategy) {
-			setChainingStrategy(chainingStrategy);
-		}
-
-		@Override
-		public void processElement1(StreamRecord<String> element) throws Exception {
-			if (!semaphore.tryAcquire()) {
-				Assert.fail("Concurrent invocation of operator functions.");
-			}
-
-			if (first) {
-				registerTimer(System.currentTimeMillis() + 100, this);
-				first = false;
-			}
-			numElements++;
-
-			semaphore.release();
-		}
-
-		@Override
-		public void processElement2(StreamRecord<String> element) throws Exception {
-			if (!semaphore.tryAcquire()) {
-				Assert.fail("Concurrent invocation of operator functions.");
-			}
-
-			if (first) {
-				registerTimer(System.currentTimeMillis() + 100, this);
-				first = false;
-			}
-			numElements++;
-
-			semaphore.release();
-		}
-
-
-		@Override
-		public void trigger(long time) throws Exception {
-			if (!semaphore.tryAcquire()) {
-				Assert.fail("Concurrent invocation of operator functions.");
-			}
-
-			try {
-				numTimers++;
-				throwIfDone();
-				registerTimer(System.currentTimeMillis() + 1, this);
-			} finally {
-				semaphore.release();
-			}
-		}
-
-		private void throwIfDone() {
-			if (numTimers > 1000 && numElements > 10_000) {
-				throw new RuntimeException("TEST SUCCESS");
-			}
-		}
-
-		@Override
-		public void processWatermark1(Watermark mark) throws Exception {
-			//ignore
-		}
-
-		@Override
-		public void processWatermark2(Watermark mark) throws Exception {
-			//ignore
-		}
-	}
-
-
-	private static class InfiniteTestSource implements SourceFunction<String> {
-		private static final long serialVersionUID = 1L;
-		private volatile boolean running = true;
-
-		@Override
-		public void run(SourceContext<String> ctx) throws Exception {
-			while (running) {
-				ctx.collect("hello");
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java
deleted file mode 100644
index f87d7ea..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java
+++ /dev/null
@@ -1,373 +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.tasks;
-
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.co.RichCoMapFunction;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.co.CoStreamMap;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-/**
- * Tests for {@link org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask}. Theses tests
- * implicitly also test the {@link org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor}.
- *
- * <p>
- * Note:<br>
- * We only use a {@link CoStreamMap} operator here. We also test the individual operators but Map is
- * used as a representative to test TwoInputStreamTask, since TwoInputStreamTask is used for all
- * TwoInputStreamOperators.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ResultPartitionWriter.class})
-public class TwoInputStreamTaskTest {
-
-	/**
-	 * This test verifies that open() and close() are correctly called. This test also verifies
-	 * that timestamps of emitted elements are correct. {@link CoStreamMap} assigns the input
-	 * timestamp to emitted elements.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testOpenCloseAndTimestamps() throws Exception {
-		final TwoInputStreamTask<String, Integer, String> coMapTask = new TwoInputStreamTask<String, Integer, String>();
-		final TwoInputStreamTaskTestHarness<String, Integer, String> testHarness = new TwoInputStreamTaskTestHarness<String, Integer, String>(coMapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		CoStreamMap<String, Integer, String> coMapOperator = new CoStreamMap<String, Integer, String>(new TestOpenCloseMapFunction());
-		streamConfig.setStreamOperator(coMapOperator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.invoke();
-
-		testHarness.processElement(new StreamRecord<String>("Hello", initialTime + 1), 0, 0);
-		expectedOutput.add(new StreamRecord<String>("Hello", initialTime + 1));
-
-		// wait until the input is processed to ensure ordering of the output
-		testHarness.waitForInputProcessing();
-
-		testHarness.processElement(new StreamRecord<Integer>(1337, initialTime + 2), 1, 0);
-
-		expectedOutput.add(new StreamRecord<String>("1337", initialTime + 2));
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseMapFunction.closeCalled);
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	/**
-	 * This test verifies that watermarks are correctly forwarded. This also checks whether
-	 * watermarks are forwarded only when we have received watermarks from all inputs. The
-	 * forwarded watermark must be the minimum of the watermarks of all inputs.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testWatermarkForwarding() throws Exception {
-		final TwoInputStreamTask<String, Integer, String> coMapTask = new TwoInputStreamTask<String, Integer, String>();
-		final TwoInputStreamTaskTestHarness<String, Integer, String> testHarness = new TwoInputStreamTaskTestHarness<String, Integer, String>(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		CoStreamMap<String, Integer, String> coMapOperator = new CoStreamMap<String, Integer, String>(new IdentityMap());
-		streamConfig.setStreamOperator(coMapOperator);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-		long initialTime = 0L;
-
-		testHarness.invoke();
-
-		testHarness.processElement(new Watermark(initialTime), 0, 0);
-		testHarness.processElement(new Watermark(initialTime), 0, 1);
-
-		testHarness.processElement(new Watermark(initialTime), 1, 0);
-
-
-		// now the output should still be empty
-		testHarness.waitForInputProcessing();
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		testHarness.processElement(new Watermark(initialTime), 1, 1);
-
-		// now the watermark should have propagated, Map simply forward Watermarks
-		testHarness.waitForInputProcessing();
-		expectedOutput.add(new Watermark(initialTime));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		// contrary to checkpoint barriers these elements are not blocked by watermarks
-		testHarness.processElement(new StreamRecord<String>("Hello", initialTime), 0, 0);
-		testHarness.processElement(new StreamRecord<Integer>(42, initialTime), 1, 1);
-		expectedOutput.add(new StreamRecord<String>("Hello", initialTime));
-		expectedOutput.add(new StreamRecord<String>("42", initialTime));
-
-		testHarness.waitForInputProcessing();
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		testHarness.processElement(new Watermark(initialTime + 4), 0, 0);
-		testHarness.processElement(new Watermark(initialTime + 3), 0, 1);
-		testHarness.processElement(new Watermark(initialTime + 3), 1, 0);
-		testHarness.processElement(new Watermark(initialTime + 2), 1, 1);
-
-		// check whether we get the minimum of all the watermarks, this must also only occur in
-		// the output after the two StreamRecords
-		expectedOutput.add(new Watermark(initialTime + 2));
-		testHarness.waitForInputProcessing();
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-
-		// advance watermark from one of the inputs, now we should get a now one since the
-		// minimum increases
-		testHarness.processElement(new Watermark(initialTime + 4), 1, 1);
-		testHarness.waitForInputProcessing();
-		expectedOutput.add(new Watermark(initialTime + 3));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		// advance the other two inputs, now we should get a new one since the
-		// minimum increases again
-		testHarness.processElement(new Watermark(initialTime + 4), 0, 1);
-		testHarness.processElement(new Watermark(initialTime + 4), 1, 0);
-		testHarness.waitForInputProcessing();
-		expectedOutput.add(new Watermark(initialTime + 4));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		List<String> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput());
-		Assert.assertEquals(2, resultElements.size());
-	}
-
-	/**
-	 * This test verifies that checkpoint barriers are correctly forwarded.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testCheckpointBarriers() throws Exception {
-		final TwoInputStreamTask<String, Integer, String> coMapTask = new TwoInputStreamTask<String, Integer, String>();
-		final TwoInputStreamTaskTestHarness<String, Integer, String> testHarness = new TwoInputStreamTaskTestHarness<String, Integer, String>(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		CoStreamMap<String, Integer, String> coMapOperator = new CoStreamMap<String, Integer, String>(new IdentityMap());
-		streamConfig.setStreamOperator(coMapOperator);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-		long initialTime = 0L;
-
-		testHarness.invoke();
-
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0);
-
-		// This element should be buffered since we received a checkpoint barrier on
-		// this input
-		testHarness.processElement(new StreamRecord<String>("Hello-0-0", initialTime), 0, 0);
-
-		// This one should go through
-		testHarness.processElement(new StreamRecord<String>("Ciao-0-0", initialTime), 0, 1);
-		expectedOutput.add(new StreamRecord<String>("Ciao-0-0", initialTime));
-
-		// These elements should be forwarded, since we did not yet receive a checkpoint barrier
-		// on that input, only add to same input, otherwise we would not know the ordering
-		// of the output since the Task might read the inputs in any order
-		testHarness.processElement(new StreamRecord<Integer>(11, initialTime), 1, 1);
-		testHarness.processElement(new StreamRecord<Integer>(111, initialTime), 1, 1);
-		expectedOutput.add(new StreamRecord<String>("11", initialTime));
-		expectedOutput.add(new StreamRecord<String>("111", initialTime));
-
-		testHarness.waitForInputProcessing();
-		// we should not yet see the barrier, only the two elements from non-blocked input
-		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				testHarness.getOutput(),
-				expectedOutput);
-
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1);
-
-		testHarness.waitForInputProcessing();
-
-		// now we should see the barrier and after that the buffered elements
-		expectedOutput.add(new CheckpointBarrier(0, 0));
-		expectedOutput.add(new StreamRecord<String>("Hello-0-0", initialTime));
-		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				testHarness.getOutput(),
-				expectedOutput);
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		List<String> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput());
-		Assert.assertEquals(4, resultElements.size());
-	}
-
-	/**
-	 * This test verifies that checkpoint barriers and barrier buffers work correctly with
-	 * concurrent checkpoint barriers where one checkpoint is "overtaking" another checkpoint, i.e.
-	 * some inputs receive barriers from an earlier checkpoint, thereby blocking,
-	 * then all inputs receive barriers from a later checkpoint.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testOvertakingCheckpointBarriers() throws Exception {
-		final TwoInputStreamTask<String, Integer, String> coMapTask = new TwoInputStreamTask<String, Integer, String>();
-		final TwoInputStreamTaskTestHarness<String, Integer, String> testHarness = new TwoInputStreamTaskTestHarness<String, Integer, String>(coMapTask, 2, 2, new int[] {1, 2}, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		CoStreamMap<String, Integer, String> coMapOperator = new CoStreamMap<String, Integer, String>(new IdentityMap());
-		streamConfig.setStreamOperator(coMapOperator);
-
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-		long initialTime = 0L;
-
-		testHarness.invoke();
-
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 0);
-
-		// These elements should be buffered until we receive barriers from
-		// all inputs
-		testHarness.processElement(new StreamRecord<String>("Hello-0-0", initialTime), 0, 0);
-		testHarness.processElement(new StreamRecord<String>("Ciao-0-0", initialTime), 0, 0);
-
-		// These elements should be forwarded, since we did not yet receive a checkpoint barrier
-		// on that input, only add to same input, otherwise we would not know the ordering
-		// of the output since the Task might read the inputs in any order
-		testHarness.processElement(new StreamRecord<Integer>(42, initialTime), 1, 1);
-		testHarness.processElement(new StreamRecord<Integer>(1337, initialTime), 1, 1);
-		expectedOutput.add(new StreamRecord<String>("42", initialTime));
-		expectedOutput.add(new StreamRecord<String>("1337", initialTime));
-
-		testHarness.waitForInputProcessing();
-		// we should not yet see the barrier, only the two elements from non-blocked input
-		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				expectedOutput,
-				testHarness.getOutput());
-
-		// Now give a later barrier to all inputs, this should unblock the first channel,
-		// thereby allowing the two blocked elements through
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 0);
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 0, 1);
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 0);
-		testHarness.processEvent(new CheckpointBarrier(1, 1), 1, 1);
-
-		expectedOutput.add(new StreamRecord<String>("Hello-0-0", initialTime));
-		expectedOutput.add(new StreamRecord<String>("Ciao-0-0", initialTime));
-		expectedOutput.add(new CheckpointBarrier(1, 1));
-
-		testHarness.waitForInputProcessing();
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				expectedOutput,
-				testHarness.getOutput());
-
-
-		// Then give the earlier barrier, these should be ignored
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 0, 1);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 0);
-		testHarness.processEvent(new CheckpointBarrier(0, 0), 1, 1);
-
-		testHarness.waitForInputProcessing();
-
-
-		testHarness.endInput();
-
-		testHarness.waitForTaskCompletion();
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.",
-				expectedOutput,
-				testHarness.getOutput());
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseMapFunction extends RichCoMapFunction<String, Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public String map1(String value) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return value;
-		}
-
-		@Override
-		public String map2(Integer value) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return value.toString();
-		}
-	}
-
-	private static class IdentityMap implements CoMapFunction<String, Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map1(String value) throws Exception {
-			return value;
-		}
-
-		@Override
-		public String map2(Integer value) throws Exception {
-
-			return value.toString();
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
deleted file mode 100644
index 2b20101..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.tasks;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.graph.StreamNode;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
-
-/**
- * Test harness for testing a {@link TwoInputStreamTask}.
- *
- * <p>
- * This mock Invokable provides the task with a basic runtime context and allows pushing elements
- * and watermarks into the task. {@link #getOutput()} can be used to get the emitted elements
- * and events. You are free to modify the retrieved list.
- *
- * <p>
- * After setting up everything the Task can be invoked using {@link #invoke()}. This will start
- * a new Thread to execute the Task. Use {@link #waitForTaskCompletion()} to wait for the Task
- * thread to finish. Use {@link #processElement}
- * to send elements to the task. Use
- * {@link #processEvent(org.apache.flink.runtime.event.AbstractEvent)} to send events to the task.
- * Before waiting for the task to finish you must call {@link #endInput()} to signal to the task
- * that data entry is finished.
- *
- * <p>
- * When Elements or Events are offered to the Task they are put into a queue. The input gates
- * of the Task read from this queue. Use {@link #waitForInputProcessing()} to wait until all
- * queues are empty. This must be used after entering some elements before checking the
- * desired output.
- *
- * <p>
- * When using this you need to add the following line to your test class to setup Powermock:
- * {@code @PrepareForTest({ResultPartitionWriter.class})}
- */
-public class TwoInputStreamTaskTestHarness<IN1, IN2, OUT> extends StreamTaskTestHarness<OUT> {
-
-	private TypeInformation<IN1> inputType1;
-	private TypeSerializer<IN1> inputSerializer1;
-
-	private TypeInformation<IN2> inputType2;
-	private TypeSerializer<IN2> inputSerializer2;
-
-	private int[] inputGateAssignment;
-
-	/**
-	 * Creates a test harness with the specified number of input gates and specified number
-	 * of channels per input gate. Parameter inputGateAssignment specifies for each gate whether
-	 * it should be assigned to the first (1), or second (2) input of the task.
-	 */
-	public TwoInputStreamTaskTestHarness(TwoInputStreamTask<IN1, IN2, OUT> task,
-			int numInputGates,
-			int numInputChannelsPerGate,
-			int[] inputGateAssignment,
-			TypeInformation<IN1> inputType1,
-			TypeInformation<IN2> inputType2,
-			TypeInformation<OUT> outputType) {
-		super(task, outputType);
-
-		this.inputType1 = inputType1;
-		inputSerializer1 = inputType1.createSerializer(executionConfig);
-
-		this.inputType2 = inputType2;
-		inputSerializer2 = inputType2.createSerializer(executionConfig);
-
-		this.numInputGates = numInputGates;
-		this.numInputChannelsPerGate = numInputChannelsPerGate;
-		this.inputGateAssignment = inputGateAssignment;
-	}
-
-	/**
-	 * Creates a test harness with one input gate (that has one input channel) per input. The first
-	 * input gate is assigned to the first task input, the second input gate is assigned to the
-	 * second task input.
-	 */
-	public TwoInputStreamTaskTestHarness(TwoInputStreamTask<IN1, IN2, OUT> task,
-			TypeInformation<IN1> inputType1,
-			TypeInformation<IN2> inputType2,
-			TypeInformation<OUT> outputType) {
-		this(task, 2, 1, new int[] {1, 2}, inputType1, inputType2, outputType);
-	}
-
-	@Override
-	protected void initializeInputs() throws IOException, InterruptedException {
-
-		inputGates = new StreamTestSingleInputGate[numInputGates];
-		List<StreamEdge> inPhysicalEdges = new LinkedList<StreamEdge>();
-
-		StreamOperator<IN1> dummyOperator = new AbstractStreamOperator<IN1>() {
-			private static final long serialVersionUID = 1L;
-		};
-
-		StreamNode sourceVertexDummy = new StreamNode(null, 0, dummyOperator, "source dummy", new LinkedList<OutputSelector<?>>(), SourceStreamTask.class);
-		StreamNode targetVertexDummy = new StreamNode(null, 1, dummyOperator, "target dummy", new LinkedList<OutputSelector<?>>(), SourceStreamTask.class);
-
-		for (int i = 0; i < numInputGates; i++) {
-
-			switch (inputGateAssignment[i]) {
-				case 1: {
-					inputGates[i] = new StreamTestSingleInputGate<IN1>(
-							numInputChannelsPerGate,
-							bufferSize,
-							inputSerializer1);
-
-
-					StreamEdge streamEdge = new StreamEdge(sourceVertexDummy,
-							targetVertexDummy,
-							1,
-							new LinkedList<String>(),
-							new BroadcastPartitioner<Object>());
-
-					inPhysicalEdges.add(streamEdge);
-					break;
-				}
-				case 2: {
-					inputGates[i] = new StreamTestSingleInputGate<IN2>(
-							numInputChannelsPerGate,
-							bufferSize,
-							inputSerializer2);
-
-					StreamEdge streamEdge = new StreamEdge(sourceVertexDummy,
-							targetVertexDummy,
-							2,
-							new LinkedList<String>(),
-							new BroadcastPartitioner<Object>());
-
-					inPhysicalEdges.add(streamEdge);
-					break;
-				}
-				default:
-					throw new IllegalStateException("Wrong input gate assignment.");
-			}
-
-			this.mockEnv.addInputGate(inputGates[i].getInputGate());
-		}
-
-		streamConfig.setInPhysicalEdges(inPhysicalEdges);
-		streamConfig.setNumberOfInputs(numInputGates);
-		streamConfig.setTypeSerializerIn1(inputSerializer1);
-		streamConfig.setTypeSerializerIn2(inputSerializer2);
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
deleted file mode 100644
index 749e1dd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
+++ /dev/null
@@ -1,619 +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.timestamp;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.taskmanager.MultiShotLatch;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.AscendingTimestampExtractor;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.NoOpSink;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.fail;
-
-/**
- * Tests for timestamps, watermarks, and event-time sources.
- */
-@SuppressWarnings("serial")
-public class TimestampITCase {
-
-	private static final int NUM_TASK_MANAGERS = 2;
-	private static final int NUM_TASK_SLOTS = 3;
-	private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS;
-
-	// this is used in some tests to synchronize
-	static MultiShotLatch latch;
-
-
-	private static ForkableFlinkMiniCluster cluster;
-
-	@Before
-	public void setupLatch() {
-		// ensure that we get a fresh latch for each test
-		latch = new MultiShotLatch();
-	}
-
-
-	@BeforeClass
-	public static void startCluster() {
-		try {
-			Configuration config = new Configuration();
-			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS);
-			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
-			config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms");
-			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
-
-			cluster = new ForkableFlinkMiniCluster(config, false);
-
-			cluster.start();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to start test cluster: " + e.getMessage());
-		}
-	}
-
-	@AfterClass
-	public static void shutdownCluster() {
-		try {
-			cluster.shutdown();
-			cluster = null;
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Failed to stop test cluster: " + e.getMessage());
-		}
-	}
-
-	/**
-	 * These check whether custom timestamp emission works at sources and also whether timestamps
-	 * arrive at operators throughout a topology.
-	 *
-	 * <p>
-	 * This also checks whether watermarks keep propagating if a source closes early.
-	 *
-	 * <p>
-	 * This only uses map to test the workings of watermarks in a complete, running topology. All
-	 * tasks and stream operators have dedicated tests that test the watermark propagation
-	 * behaviour.
-	 */
-	@Test
-	public void testWatermarkPropagation() throws Exception {
-		final int NUM_WATERMARKS = 10;
-
-		long initialTime = 0L;
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-		env.getConfig().enableTimestamps();
-
-
-		DataStream<Integer> source1 = env.addSource(new MyTimestampSource(initialTime, NUM_WATERMARKS));
-		DataStream<Integer> source2 = env.addSource(new MyTimestampSource(initialTime, NUM_WATERMARKS / 2));
-
-		source1.union(source2)
-				.map(new IdentityMap())
-				.connect(source2).map(new IdentityCoMap())
-				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator())
-				.addSink(new NoOpSink<Integer>());
-
-		env.execute();
-
-		// verify that all the watermarks arrived at the final custom operator
-		for (int i = 0; i < PARALLELISM; i++) {
-			// There can be two cases, either we get NUM_WATERMARKS + 1 watermarks or
-			// (NUM_WATERMARKS / 2) + 1 watermarks. This depends on which source get's to run first.
-			// If source1 runs first we jump directly to +Inf and skip all the intermediate
-			// watermarks. If source2 runs first we see the intermediate watermarks from
-			// NUM_WATERMARKS/2 to +Inf.
-			if (CustomOperator.finalWatermarks[i].size() == NUM_WATERMARKS + 1) {
-				for (int j = 0; j < NUM_WATERMARKS; j++) {
-					if (!CustomOperator.finalWatermarks[i].get(j).equals(new Watermark(initialTime + j))) {
-						System.err.println("All Watermarks: ");
-						for (int k = 0; k <= NUM_WATERMARKS; k++) {
-							System.err.println(CustomOperator.finalWatermarks[i].get(k));
-						}
-
-						Assert.fail("Wrong watermark.");
-					}
-				}
-				if (!CustomOperator.finalWatermarks[i].get(NUM_WATERMARKS).equals(new Watermark(Long.MAX_VALUE))) {
-					System.err.println("All Watermarks: ");
-					for (int k = 0; k <= NUM_WATERMARKS; k++) {
-						System.err.println(CustomOperator.finalWatermarks[i].get(k));
-					}
-
-					Assert.fail("Wrong watermark.");
-				}
-			} else {
-				for (int j = 0; j < NUM_WATERMARKS / 2; j++) {
-					if (!CustomOperator.finalWatermarks[i].get(j).equals(new Watermark(initialTime + j))) {
-						System.err.println("All Watermarks: ");
-						for (int k = 0; k <= NUM_WATERMARKS / 2; k++) {
-							System.err.println(CustomOperator.finalWatermarks[i].get(k));
-						}
-
-						Assert.fail("Wrong watermark.");
-					}
-				}
-				if (!CustomOperator.finalWatermarks[i].get(NUM_WATERMARKS / 2).equals(new Watermark(Long.MAX_VALUE))) {
-					System.err.println("All Watermarks: ");
-					for (int k = 0; k <= NUM_WATERMARKS / 2; k++) {
-						System.err.println(CustomOperator.finalWatermarks[i].get(k));
-					}
-
-					Assert.fail("Wrong watermark.");
-				}
-
-			}
-
-		}
-	}
-
-
-
-	/**
-	 * These check whether timestamps are properly assigned at the sources and handled in
-	 * network transmission and between chained operators when timestamps are enabled.
-	 */
-	@Test
-	public void testTimestampHandling() throws Exception {
-		final int NUM_ELEMENTS = 10;
-
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-		env.getConfig().enableTimestamps();
-
-
-		DataStream<Integer> source1 = env.addSource(new MyTimestampSource(0L, NUM_ELEMENTS));
-		DataStream<Integer> source2 = env.addSource(new MyTimestampSource(0L, NUM_ELEMENTS));
-
-		source1
-				.map(new IdentityMap())
-				.connect(source2).map(new IdentityCoMap())
-				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator())
-				.addSink(new NoOpSink<Integer>());
-
-
-		env.execute();
-	}
-
-	/**
-	 * These check whether timestamps are properly ignored when they are disabled.
-	 */
-	@Test
-	public void testDisabledTimestamps() throws Exception {
-		final int NUM_ELEMENTS = 10;
-
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
-				"localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-		Assert.assertEquals("Timestamps are not disabled by default.",
-				false,
-				env.getConfig().areTimestampsEnabled());
-		env.getConfig().disableTimestamps();
-
-
-		DataStream<Integer> source1 = env.addSource(new MyNonWatermarkingSource(NUM_ELEMENTS));
-		DataStream<Integer> source2 = env.addSource(new MyNonWatermarkingSource(NUM_ELEMENTS));
-
-		source1
-				.map(new IdentityMap())
-				.connect(source2).map(new IdentityCoMap())
-				.transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new DisabledTimestampCheckingOperator())
-				.addSink(new NoOpSink<Integer>());
-
-
-		env.execute();
-	}
-
-	/**
-	 * This thests whether timestamps are properly extracted in the timestamp
-	 * extractor and whether watermarks are also correctly forwared from this with the auto watermark
-	 * interval.
-	 */
-	@Test
-	public void testTimestampExtractorWithAutoInterval() throws Exception {
-		final int NUM_ELEMENTS = 10;
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(1);
-		env.getConfig().disableSysoutLogging();
-		env.getConfig().enableTimestamps();
-		env.getConfig().setAutoWatermarkInterval(10);
-
-
-		DataStream<Integer> source1 = env.addSource(new SourceFunction<Integer>() {
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-				int index = 0;
-				while (index < NUM_ELEMENTS) {
-					ctx.collect(index);
-					latch.await();
-					index++;
-				}
-			}
-
-			@Override
-			public void cancel() {
-
-			}
-		});
-
-		DataStream<Integer> extractOp = source1.assignTimestamps(
-				new AscendingTimestampExtractor<Integer>() {
-					@Override
-					public long extractAscendingTimestamp(Integer element, long currentTimestamp) {
-						return element;
-					}
-				});
-
-		extractOp
-				.transform("Watermark Check", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator())
-				.transform("Timestamp Check",
-						BasicTypeInfo.INT_TYPE_INFO,
-						new TimestampCheckingOperator());
-
-		// verify that extractor picks up source parallelism
-		Assert.assertEquals(extractOp.getTransformation().getParallelism(), source1.getTransformation().getParallelism());
-
-		env.execute();
-
-		// verify that we get NUM_ELEMENTS watermarks
-		for (int j = 0; j < NUM_ELEMENTS; j++) {
-			if (!CustomOperator.finalWatermarks[0].get(j).equals(new Watermark(j - 1))) {
-				Assert.fail("Wrong watermark.");
-			}
-		}
-		if (!CustomOperator.finalWatermarks[0].get(NUM_ELEMENTS).equals(new Watermark(Long.MAX_VALUE))) {
-			Assert.fail("Wrong watermark.");
-		}
-	}
-
-	/**
-	 * This thests whether timestamps are properly extracted in the timestamp
-	 * extractor and whether watermark are correctly forwarded from the custom watermark emit
-	 * function.
-	 */
-	@Test
-	public void testTimestampExtractorWithCustomWatermarkEmit() throws Exception {
-		final int NUM_ELEMENTS = 10;
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(1);
-		env.getConfig().disableSysoutLogging();
-		env.getConfig().enableTimestamps();
-
-
-		DataStream<Integer> source1 = env.addSource(new SourceFunction<Integer>() {
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-				int index = 0;
-				while (index < NUM_ELEMENTS) {
-					ctx.collect(index);
-					latch.await();
-					index++;
-				}
-			}
-
-			@Override
-			public void cancel() {
-
-			}
-		});
-
-		source1.assignTimestamps(new TimestampExtractor<Integer>() {
-			@Override
-			public long extractTimestamp(Integer element, long currentTimestamp) {
-				return element;
-			}
-
-			@Override
-			public long extractWatermark(Integer element, long currentTimestamp) {
-				return element - 1;
-			}
-
-			@Override
-			public long getCurrentWatermark() {
-				return Long.MIN_VALUE;
-			}
-		})
-				.transform("Watermark Check", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator())
-				.transform("Timestamp Check", BasicTypeInfo.INT_TYPE_INFO, new TimestampCheckingOperator());
-
-
-		env.execute();
-
-		// verify that we get NUM_ELEMENTS watermarks
-		for (int j = 0; j < NUM_ELEMENTS; j++) {
-			if (!CustomOperator.finalWatermarks[0].get(j).equals(new Watermark(j - 1))) {
-				Assert.fail("Wrong watermark.");
-			}
-		}
-		if (!CustomOperator.finalWatermarks[0].get(NUM_ELEMENTS).equals(new Watermark(Long.MAX_VALUE))) {
-			Assert.fail("Wrong watermark.");
-		}
-	}
-
-	/**
-	 * This tests whether the program throws an exception when an event-time source tries
-	 * to emit without timestamp.
-	 */
-	@Test(expected = ProgramInvocationException.class)
-	public void testEventTimeSourceEmitWithoutTimestamp() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		DataStream<Integer> source1 = env.addSource(new MyErroneousTimestampSource());
-
-		source1
-				.map(new IdentityMap())
-				.addSink(new NoOpSink<Integer>());
-
-		env.execute();
-	}
-
-	/**
-	 * This tests whether the program throws an exception when a regular source tries
-	 * to emit with timestamp.
-	 */
-	@Test(expected = ProgramInvocationException.class)
-	public void testSourceEmitWithTimestamp() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		DataStream<Integer> source1 = env.addSource(new MyErroneousSource());
-
-		source1
-				.map(new IdentityMap())
-				.addSink(new NoOpSink<Integer>());
-
-		env.execute();
-	}
-
-	/**
-	 * This tests whether the program throws an exception when a regular source tries
-	 * to emit a watermark.
-	 */
-	@Test(expected = ProgramInvocationException.class)
-	public void testSourceEmitWatermark() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", cluster.getLeaderRPCPort());
-		env.setParallelism(PARALLELISM);
-		env.getConfig().disableSysoutLogging();
-
-		DataStream<Integer> source1 = env.addSource(new MyErroneousWatermarkSource());
-
-		source1
-				.map(new IdentityMap())
-				.addSink(new NoOpSink<Integer>());
-
-		env.execute();
-	}
-
-	@SuppressWarnings("unchecked")
-	public static class CustomOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
-
-		List<Watermark> watermarks;
-		public static List<Watermark>[] finalWatermarks = new List[PARALLELISM];
-		private long oldTimestamp;
-
-		@Override
-		public void processElement(StreamRecord<Integer> element) throws Exception {
-			if (element.getTimestamp() != element.getValue()) {
-				Assert.fail("Timestamps are not properly handled.");
-			}
-			oldTimestamp = element.getTimestamp();
-			output.collect(element);
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-			watermarks.add(mark);
-			latch.trigger();
-			output.emitWatermark(mark);
-		}
-
-		@Override
-		public void open() throws Exception {
-			super.open();
-			watermarks = new ArrayList<Watermark>();
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			finalWatermarks[getRuntimeContext().getIndexOfThisSubtask()] = watermarks;
-		}
-	}
-
-	public static class TimestampCheckingOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
-
-		@Override
-		public void processElement(StreamRecord<Integer> element) throws Exception {
-			if (element.getTimestamp() != element.getValue()) {
-				Assert.fail("Timestamps are not properly handled.");
-			}
-			output.collect(element);
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-		}
-	}
-
-	public static class DisabledTimestampCheckingOperator extends AbstractStreamOperator<Integer> implements OneInputStreamOperator<Integer, Integer> {
-
-		@Override
-		public void processElement(StreamRecord<Integer> element) throws Exception {
-			if (element.getTimestamp() != 0) {
-				Assert.fail("Timestamps are not properly handled.");
-			}
-			output.collect(element);
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-		}
-	}
-
-	public static class IdentityCoMap implements CoMapFunction<Integer, Integer, Integer> {
-		@Override
-		public Integer map1(Integer value) throws Exception {
-			return value;
-		}
-
-		@Override
-		public Integer map2(Integer value) throws Exception {
-			return value;
-		}
-	}
-
-	public static class IdentityMap implements MapFunction<Integer, Integer> {
-		@Override
-		public Integer map(Integer value) throws Exception {
-			return value;
-		}
-	}
-
-	public static class MyTimestampSource implements EventTimeSourceFunction<Integer> {
-
-		long initialTime;
-		int numWatermarks;
-
-		public MyTimestampSource(long initialTime, int numWatermarks) {
-			this.initialTime = initialTime;
-			this.numWatermarks = numWatermarks;
-		}
-
-		@Override
-		public void run(SourceContext<Integer> ctx) throws Exception {
-			for (int i = 0; i < numWatermarks; i++) {
-				ctx.collectWithTimestamp(i, initialTime + i);
-				ctx.emitWatermark(new Watermark(initialTime + i));
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-
-	public static class MyNonWatermarkingSource implements SourceFunction<Integer> {
-
-		int numWatermarks;
-
-		public MyNonWatermarkingSource(int numWatermarks) {
-			this.numWatermarks = numWatermarks;
-		}
-
-		@Override
-		public void run(SourceContext<Integer> ctx) throws Exception {
-			for (int i = 0; i < numWatermarks; i++) {
-				ctx.collect(i);
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-
-	// This is a event-time source. This should only emit elements with timestamps. The test should
-	// therefore throw an exception
-	public static class MyErroneousTimestampSource implements EventTimeSourceFunction<Integer> {
-
-		@Override
-		public void run(SourceContext<Integer> ctx) throws Exception {
-			for (int i = 0; i < 10; i++) {
-				ctx.collect(i);
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-
-	// This is a normal source. This should only emit elements without timestamps. The test should
-	// therefore throw an exception
-	public static class MyErroneousSource implements SourceFunction<Integer> {
-
-		@Override
-		public void run(SourceContext<Integer> ctx) throws Exception {
-			for (int i = 0; i < 10; i++) {
-				ctx.collectWithTimestamp(i, 0L);
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-
-	// This is a normal source. This should only emit elements without timestamps. This also
-	// must not emit watermarks. The test should therefore throw an exception
-	public static class MyErroneousWatermarkSource implements SourceFunction<Integer> {
-
-		@Override
-		public void run(SourceContext<Integer> ctx) throws Exception {
-			for (int i = 0; i < 10; i++) {
-				ctx.collect(i);
-				ctx.emitWatermark(new Watermark(0L));
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-}


[07/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/pom.xml b/flink-streaming-connectors/flink-connector-twitter/pom.xml
new file mode 100644
index 0000000..df963a3
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/pom.xml
@@ -0,0 +1,97 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-streaming-connectors-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-twitter</artifactId>
+	<name>flink-connector-twitter</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.twitter</groupId>
+			<artifactId>hbc-core</artifactId>
+			<version>2.2.0</version>
+		</dependency>
+		
+		<dependency>
+			<groupId>org.apache.sling</groupId>
+			<artifactId>org.apache.sling.commons.json</artifactId>
+			<version>2.0.6</version>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<!-- Override artifactSet configuration to build fat-jar with all dependencies packed. -->
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<configuration>
+							<artifactSet>
+								<includes combine.children="append">
+									<!-- We include all dependencies that transitively depend on guava -->
+									<include>com.twitter:hbc-core</include>
+									<include>com.twitter:joauth</include>
+								</includes>
+							</artifactSet>
+							<transformers>
+								<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
+							</transformers>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
new file mode 100644
index 0000000..0f16541
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.json;
+
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.sling.commons.json.JSONException;
+
+/**
+ * Abstract class derived from {@link RichFlatMapFunction} to handle JSON files.
+ * 
+ * @param <IN>
+ *            Type of the input elements.
+ * @param <OUT>
+ *            Type of the returned elements.
+ */
+public abstract class JSONParseFlatMap<IN, OUT> extends RichFlatMapFunction<IN, OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	// private static final Log LOG = LogFactory.getLog(JSONParseFlatMap.class);
+
+	/**
+	 * Get the value object associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public Object get(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).get("retValue");
+	}
+
+	/**
+	 * Get the boolean value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public boolean getBoolean(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getBoolean("retValue");
+	}
+
+	/**
+	 * Get the double value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public double getDouble(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getDouble("retValue");
+	}
+
+	/**
+	 * Get the int value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public int getInt(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getInt("retValue");
+	}
+
+	/**
+	 * Get the long value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public long getLong(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+
+		return parser.parse(field).getLong("retValue");
+	}
+	
+	/**
+	 * Get the String value associated with a key form a JSON code. It can find
+	 * embedded fields, too.
+	 * 
+	 * @param jsonText
+	 *            JSON String in which the field is searched.
+	 * @param field
+	 *            The key whose value is searched for.
+	 * @return The object associated with the field.
+	 * @throws JSONException
+	 *             If the field is not found.
+	 */
+	public String getString(String jsonText, String field) throws JSONException {
+		JSONParser parser = new JSONParser(jsonText);
+		
+		return parser.parse(field).getString("retValue");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
new file mode 100644
index 0000000..c1eabbd
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.json;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.StringTokenizer;
+
+import org.apache.sling.commons.json.JSONArray;
+import org.apache.sling.commons.json.JSONException;
+import org.apache.sling.commons.json.JSONObject;
+
+/**
+ * A JSONParser contains a JSONObject and provides opportunity to access
+ * embedded fields in JSON code.
+ */
+public class JSONParser {
+
+	private JSONObject originalJO;
+	private String searchedfield;
+	private Object temp;
+
+	/**
+	 * Construct a JSONParser from a string. The string has to be a JSON code
+	 * from which we want to get a field.
+	 * 
+	 * @param jsonText
+	 *            A string which contains a JSON code. String representation of
+	 *            a JSON code.
+	 * @throws JSONException
+	 *             If there is a syntax error in the source string.
+	 */
+	public JSONParser(String jsonText) throws JSONException {
+		originalJO = new JSONObject(jsonText);
+	}
+
+	/**
+	 * 
+	 * Parse the JSON code passed to the constructor to find the given key.
+	 * 
+	 * @param key
+	 *            The key whose value is searched for.
+	 * @return A JSONObject which has only one field called "retValue" and the
+	 *         value associated to it is the searched value. The methods of
+	 *         JSONObject can be used to get the field value in a desired
+	 *         format.
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	public JSONObject parse(String key) throws JSONException {
+		initializeParser(key);
+		parsing();
+		return putResultInJSONObj();
+	}
+
+	/**
+	 * Prepare the fields of the class for the parsing
+	 * 
+	 * @param key
+	 *            The key whose value is searched for.
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	private void initializeParser(String key) throws JSONException {
+		searchedfield = key;
+		temp = new JSONObject(originalJO.toString());
+	}
+
+	/**
+	 * This function goes through the given field and calls the appropriate
+	 * functions to treat the units between the punctuation marks.
+	 * 
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	private void parsing() throws JSONException {
+		StringTokenizer st = new StringTokenizer(searchedfield, ".");
+		while (st.hasMoreTokens()) {
+			find(st.nextToken());
+		}
+	}
+
+	/**
+	 * Search for the next part of the field and update the state if it was
+	 * found.
+	 * 
+	 * @param nextToken
+	 *            The current part of the searched field.
+	 * @throws JSONException
+	 *             If the key is not found.
+	 */
+	private void find(String nextToken) throws JSONException {
+		if (endsWithBracket(nextToken)) {
+			treatAllBracket(nextToken);
+		} else {
+			temp = ((JSONObject) temp).get(nextToken);
+		}
+	}
+
+	/**
+	 * Determine whether the given string ends with a closing square bracket ']'
+	 * 
+	 * @param nextToken
+	 *            The current part of the searched field.
+	 * @return True if the given string ends with a closing square bracket ']'
+	 *         and false otherwise.
+	 */
+	private boolean endsWithBracket(String nextToken) {
+		return nextToken.substring(nextToken.length() - 1).endsWith("]");
+	}
+
+	/**
+	 * Handle (multidimensional) arrays. Treat the square bracket pairs one
+	 * after the other if necessary.
+	 * 
+	 * @param nextToken
+	 *            The current part of the searched field.
+	 * @throws JSONException
+	 *             If the searched element is not found.
+	 */
+	private void treatAllBracket(String nextToken) throws JSONException {
+		List<String> list = Arrays.asList(nextToken.split("\\["));
+		ListIterator<String> iter = list.listIterator();
+
+		temp = ((JSONObject) temp).get(iter.next());
+
+		while (iter.hasNext()) {
+			int index = Integer.parseInt(cutBracket(iter.next()));
+			temp = ((JSONArray) temp).get(index);
+		}
+	}
+
+	/**
+	 * Remove the last character of the string.
+	 * 
+	 * @param string
+	 *            String to modify.
+	 * @return The given string without the last character.
+	 */
+	private String cutBracket(String string) {
+		return string.substring(0, string.length() - 1);
+	}
+
+	/**
+	 * Save the result of the search into a JSONObject.
+	 * 
+	 * @return A special JSONObject which contain only one key. The value
+	 *         associated to this key is the result of the search.
+	 * @throws JSONException
+	 *             If there is a problem creating the JSONObject. (e.g. invalid
+	 *             syntax)
+	 */
+	private JSONObject putResultInJSONObj() throws JSONException {
+		JSONObject jo = new JSONObject();
+		jo.put("retValue", temp);
+		return jo;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java
new file mode 100644
index 0000000..8dd4458
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java
@@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.twitter;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.twitter.hbc.core.endpoint.Location;
+import com.twitter.hbc.core.endpoint.StatusesFilterEndpoint;
+import com.twitter.hbc.httpclient.auth.Authentication;
+
+/**
+ * 
+ * An extension of {@link TwitterSource} by filter parameters. This extension
+ * enables to filter the twitter stream by user defined parameters.
+ */
+public class TwitterFilterSource extends TwitterSource {
+
+	private static final Logger LOG = LoggerFactory
+			.getLogger(TwitterFilterSource.class);
+
+	private static final long serialVersionUID = 1L;
+
+	private List<String> trackTerms = new LinkedList<String>();
+
+	private List<String> languages = new LinkedList<String>();
+
+	private List<Long> followings = new LinkedList<Long>();
+
+	private List<Location> locations = new LinkedList<Location>();
+
+	private Map<String, String> queryParameters = new HashMap<String, String>();
+
+	private Map<String, String> postParameters = new HashMap<String, String>();
+
+	public TwitterFilterSource(String authPath) {
+		super(authPath);
+	}
+
+	@Override
+	protected void initializeConnection() {
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Initializing Twitter Streaming API connection");
+		}
+		queue = new LinkedBlockingQueue<String>(queueSize);
+
+		StatusesFilterEndpoint endpoint = new StatusesFilterEndpoint();
+		configEndpoint(endpoint);
+		endpoint.stallWarnings(false);
+
+		Authentication auth = authenticate();
+
+		initializeClient(endpoint, auth);
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Twitter Streaming API connection established successfully");
+		}
+	}
+
+	/**
+	 * This function configures the streaming endpoint
+	 * 
+	 * @param endpoint
+	 *            The streaming endpoint
+	 */
+	private void configEndpoint(StatusesFilterEndpoint endpoint) {
+		if (!trackTerms.isEmpty()) {
+			endpoint.trackTerms(trackTerms);
+		}
+		if (!languages.isEmpty()) {
+			endpoint.languages(languages);
+		}
+		if (!followings.isEmpty()) {
+			endpoint.followings(followings);
+		}
+		if (!locations.isEmpty()) {
+			endpoint.locations(locations);
+		}
+		if (!queryParameters.isEmpty()) {
+			for (Entry<String, String> entry : queryParameters.entrySet()) {
+				endpoint.addQueryParameter(entry.getKey(), entry.getValue());
+			}
+		}
+		if (!postParameters.isEmpty()) {
+			for (Entry<String, String> entry : postParameters.entrySet()) {
+				endpoint.addPostParameter(entry.getKey(), entry.getValue());
+			}
+		}
+	}
+
+	/**
+	 * This function sets which term to track.
+	 * 
+	 * @param term
+	 *            The term to track.
+	 */
+	public void trackTerm(String term) {
+		this.trackTerms.add(term);
+	}
+
+	/**
+	 * This function sets which terms to track.
+	 * 
+	 * @param terms
+	 *            The terms to track.
+	 */
+	public void trackTerms(Collection<String> terms) {
+		this.trackTerms.addAll(terms);
+	}
+
+	/**
+	 * This function tells which terms are tracked.
+	 */
+	public List<String> getTrackTerms() {
+		return this.trackTerms;
+	}
+
+	/**
+	 * This function sets which language to filter.
+	 * 
+	 * @param language
+	 *            The language to filter.
+	 */
+	public void filterLanguage(String language) {
+		this.languages.add(language);
+	}
+
+	/**
+	 * This function sets which languages to filter.
+	 * 
+	 * @param languages
+	 *            The languages to filter.
+	 */
+	public void filterLanguages(Collection<String> languages) {
+		this.languages.addAll(languages);
+	}
+
+	/**
+	 * This function tells which languages are filtered.
+	 */
+	public List<String> getLanguages() {
+		return this.languages;
+	}
+
+	/**
+	 * This function sets which user to follow.
+	 * 
+	 * @param userID
+	 *            The ID of the user to follow.
+	 */
+	public void filterFollowings(Long userID) {
+		this.followings.add(userID);
+	}
+
+	/**
+	 * This function sets which users to follow.
+	 * 
+	 * @param userIDs
+	 *            The IDs of the users to follow.
+	 */
+	public void filterFollowings(Collection<Long> userIDs) {
+		this.followings.addAll(userIDs);
+	}
+
+	/**
+	 * This function tells which users are followed.
+	 */
+	public List<Long> getFollowings() {
+		return this.followings;
+	}
+
+	/**
+	 * This function sets which location to filter.
+	 * 
+	 * @param location
+	 *            The location to filter.
+	 */
+	public void filterLocation(Location location) {
+		this.locations.add(location);
+	}
+
+	/**
+	 * This function sets which locations to filter.
+	 * 
+	 * @param locations
+	 *            The locations to filter.
+	 */
+	public void filterLocations(Collection<Location> locations) {
+		this.locations.addAll(locations);
+	}
+
+	/**
+	 * This function tells which locations are filtered.
+	 */
+	public List<Location> getLocations() {
+		return this.locations;
+	}
+
+	/**
+	 * This function sets a query parameter.
+	 * 
+	 * @param parameter
+	 *            The name of the query parameter.
+	 * @param value
+	 *            The value of the query parameter.
+	 */
+	public void addQueryParameter(String parameter, String value) {
+		this.queryParameters.put(parameter, value);
+	}
+
+	/**
+	 * This function sets query parameters.
+	 * 
+	 * @param queryParameters
+	 *            The query parameters for the endpoint.
+	 */
+	public void addQueryParameters(Map<String, String> queryParameters) {
+		this.queryParameters.putAll(queryParameters);
+	}
+
+	/**
+	 * This function tells which query parameters are used by the endpoint.
+	 */
+	public Map<String, String> getQueryParameters() {
+		return this.queryParameters;
+	}
+
+	/**
+	 * This function sets a post parameter.
+	 * 
+	 * @param parameter
+	 *            The name of the post parameter.
+	 * @param value
+	 *            The value of the post parameter.
+	 */
+	public void addPostParameter(String parameter, String value) {
+		this.postParameters.put(parameter, value);
+	}
+
+	/**
+	 * This function sets post parameters.
+	 * 
+	 * @param postParameters
+	 *              The post parameters for the endpoint.
+	 */
+	public void addPostParameters(Map<String, String> postParameters) {
+		this.postParameters.putAll(postParameters);
+	}
+
+	/**
+	 * This function tells which post parameters are used by the endpoint.
+	 */
+	public Map<String, String> postParameters() {
+		return this.postParameters;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.java
new file mode 100644
index 0000000..43cb179
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.twitter;
+
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.json.JSONParseFlatMap;
+import org.apache.flink.util.Collector;
+
+/**
+ * This is an example how to use TwitterFilterSource. Before executing the
+ * example you have to define the access keys of twitter.properties in the
+ * resource folder. The access keys can be found in your twitter account.
+ */
+public class TwitterFilterSourceExample {
+
+	/**
+	 * path to the twitter properties
+	 */
+	private static final String PATH_TO_AUTH_FILE = "/twitter.properties";
+
+	public static void main(String[] args) {
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment
+				.getExecutionEnvironment();
+
+		TwitterFilterSource twitterSource = new TwitterFilterSource(
+				TwitterFilterSourceExample.class.getResource(PATH_TO_AUTH_FILE)
+						.getFile());
+
+		twitterSource.trackTerm("obama");
+		twitterSource.filterLanguage("en");
+
+		DataStream<String> streamSource = env.addSource(twitterSource).flatMap(
+				new JSONParseFlatMap<String, String>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void flatMap(String s, Collector<String> c)
+							throws Exception {
+						c.collect(s);
+					}
+				});
+
+		streamSource.print();
+
+		try {
+			env.execute("Twitter Streaming Test");
+		} catch (Exception e) {
+			// TODO Auto-generated catch block
+			e.printStackTrace();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
new file mode 100644
index 0000000..bad0f8c
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.twitter;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.Properties;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.twitter.hbc.ClientBuilder;
+import com.twitter.hbc.core.Constants;
+import com.twitter.hbc.core.endpoint.DefaultStreamingEndpoint;
+import com.twitter.hbc.core.endpoint.StatusesSampleEndpoint;
+import com.twitter.hbc.core.processor.StringDelimitedProcessor;
+import com.twitter.hbc.httpclient.BasicClient;
+import com.twitter.hbc.httpclient.auth.Authentication;
+import com.twitter.hbc.httpclient.auth.OAuth1;
+
+/**
+ * Implementation of {@link SourceFunction} specialized to emit tweets from
+ * Twitter. This is not a parallel source because the Twitter API only allows
+ * two concurrent connections.
+ */
+public class TwitterSource extends RichSourceFunction<String> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TwitterSource.class);
+
+	private static final long serialVersionUID = 1L;
+	private String authPath;
+	protected transient BlockingQueue<String> queue;
+	protected int queueSize = 10000;
+	private transient BasicClient client;
+	private int waitSec = 5;
+
+	private int maxNumberOfTweets;
+	private int currentNumberOfTweets;
+
+	private transient volatile boolean isRunning;
+
+	/**
+	 * Create {@link TwitterSource} for streaming
+	 * 
+	 * @param authPath
+	 *            Location of the properties file containing the required
+	 *            authentication information.
+	 */
+	public TwitterSource(String authPath) {
+		this.authPath = authPath;
+		maxNumberOfTweets = -1;
+	}
+
+	/**
+	 * Create {@link TwitterSource} to collect finite number of tweets
+	 * 
+	 * @param authPath
+	 *            Location of the properties file containing the required
+	 *            authentication information.
+	 * @param numberOfTweets
+	 * 
+	 */
+	public TwitterSource(String authPath, int numberOfTweets) {
+		this.authPath = authPath;
+		this.maxNumberOfTweets = numberOfTweets;
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		initializeConnection();
+		currentNumberOfTweets = 0;
+		isRunning = true;
+	}
+
+	/**
+	 * Initialize Hosebird Client to be able to consume Twitter's Streaming API
+	 */
+	protected void initializeConnection() {
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Initializing Twitter Streaming API connection");
+		}
+
+		queue = new LinkedBlockingQueue<String>(queueSize);
+
+		StatusesSampleEndpoint endpoint = new StatusesSampleEndpoint();
+		endpoint.stallWarnings(false);
+
+		Authentication auth = authenticate();
+
+		initializeClient(endpoint, auth);
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Twitter Streaming API connection established successfully");
+		}
+	}
+
+	protected OAuth1 authenticate() {
+
+		Properties authenticationProperties = loadAuthenticationProperties();
+		
+		return new OAuth1(authenticationProperties.getProperty("consumerKey"),
+				authenticationProperties.getProperty("consumerSecret"),
+				authenticationProperties.getProperty("token"),
+				authenticationProperties.getProperty("secret"));
+	}
+
+	/**
+	 * Reads the given properties file for the authentication data.
+	 * 
+	 * @return the authentication data.
+	 */
+	private Properties loadAuthenticationProperties() {
+		
+		Properties properties = new Properties();
+		try {
+			InputStream input = new FileInputStream(authPath);
+			properties.load(input);
+			input.close();
+		} catch (Exception e) {
+			throw new RuntimeException("Cannot open .properties file: " + authPath, e);
+		}
+		return properties;
+	}
+
+	protected void initializeClient(DefaultStreamingEndpoint endpoint, Authentication auth) {
+
+		client = new ClientBuilder().name("twitterSourceClient").hosts(Constants.STREAM_HOST)
+				.endpoint(endpoint).authentication(auth)
+				.processor(new StringDelimitedProcessor(queue)).build();
+		
+		client.connect();
+	}
+
+	@Override
+	public void close() {
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Initiating connection close");
+		}
+
+		if (client != null) {
+			client.stop();
+		}
+
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Connection closed successfully");
+		}
+	}
+
+	/**
+	 * Get the size of the queue in which the tweets are contained temporarily.
+	 * 
+	 * @return the size of the queue in which the tweets are contained
+	 *         temporarily
+	 */
+	public int getQueueSize() {
+		return queueSize;
+	}
+
+	/**
+	 * Set the size of the queue in which the tweets are contained temporarily.
+	 * 
+	 * @param queueSize
+	 *            The desired value.
+	 */
+	public void setQueueSize(int queueSize) {
+		this.queueSize = queueSize;
+	}
+
+	/**
+	 * This function tells how long TwitterSource waits for the tweets.
+	 * 
+	 * @return Number of second.
+	 */
+	public int getWaitSec() {
+		return waitSec;
+	}
+
+	/**
+	 * This function sets how long TwitterSource should wait for the tweets.
+	 * 
+	 * @param waitSec
+	 *            The desired value.
+	 */
+	public void setWaitSec(int waitSec) {
+		this.waitSec = waitSec;
+	}
+
+	@Override
+	public void run(SourceContext<String> ctx) throws Exception {
+		while (isRunning) {
+			if (client.isDone()) {
+				if (LOG.isErrorEnabled()) {
+					LOG.error("Client connection closed unexpectedly: {}", client.getExitEvent()
+							.getMessage());
+				}
+				break;
+			}
+
+			ctx.collect(queue.take());
+
+			if (maxNumberOfTweets != -1 && currentNumberOfTweets >= maxNumberOfTweets) {
+				break;
+			}
+		}
+	}
+
+	@Override
+	public void cancel() {
+		isRunning = false;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
new file mode 100644
index 0000000..a80c32a
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
@@ -0,0 +1,99 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.flink.streaming.connectors.twitter;
+
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.connectors.json.JSONParseFlatMap;
+import org.apache.flink.util.Collector;
+import org.apache.sling.commons.json.JSONException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TwitterStreaming {
+
+	private static final int PARALLELISM = 1;
+	private static final int SOURCE_PARALLELISM = 1;
+	private static final int NUMBEROFTWEETS = 100;
+
+	private static final Logger LOG = LoggerFactory.getLogger(TwitterStreaming.class);
+
+	public static class TwitterSink implements SinkFunction<Tuple5<Long, Integer, String, String, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void invoke(Tuple5<Long, Integer, String, String, String> tuple) {
+			System.out.println("ID: " + tuple.f0 + " int: " + tuple.f1 + " LANGUAGE: " + tuple.f2);
+			System.out.println("NAME: " + tuple.f4);
+			System.out.println("TEXT: " + tuple.f3);
+			System.out.println("");
+		}
+
+	}
+
+	public static class SelectDataFlatMap extends
+			JSONParseFlatMap<String, Tuple5<Long, Integer, String, String, String>> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(String value, Collector<Tuple5<Long, Integer, String, String, String>> out)
+				throws Exception {
+			try {
+				out.collect(new Tuple5<Long, Integer, String, String, String>(
+						getLong(value, "id"),
+						getInt(value, "entities.hashtags[0].indices[1]"),
+						getString(value, "lang"),
+						getString(value, "text"),
+						getString(value, "user.name")));
+			} catch (JSONException e) {
+				if (LOG.isErrorEnabled()) {
+					LOG.error("Field not found");
+				}
+			}
+		}
+	}
+
+	public static void main(String[] args) throws Exception {
+
+		String path = new String();
+
+		if (args != null && args.length == 1) {
+			path = args[0];
+		} else {
+			System.err.println("USAGE:\nTwitterStreaming <pathToPropertiesFile>");
+			return;
+		}
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment
+				.createLocalEnvironment(PARALLELISM);
+
+		DataStream<String> streamSource = env.addSource(new TwitterSource(path, NUMBEROFTWEETS))
+				.setParallelism(SOURCE_PARALLELISM);
+
+		DataStream<Tuple5<Long, Integer, String, String, String>> selectedDataStream = streamSource
+				.flatMap(new SelectDataFlatMap());
+
+		selectedDataStream.addSink(new TwitterSink());
+
+		env.execute();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
new file mode 100644
index 0000000..b1fc92c
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
@@ -0,0 +1,92 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.flink.streaming.connectors.twitter;
+
+import org.apache.flink.api.common.functions.MapFunction;
+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.connectors.json.JSONParseFlatMap;
+import org.apache.flink.util.Collector;
+import org.apache.sling.commons.json.JSONException;
+
+/**
+* This program demonstrate the use of TwitterSource.
+* Its aim is to count the frequency of the languages of tweets
+*/
+public class TwitterTopology {
+
+	private static final int NUMBEROFTWEETS = 100;
+
+	/**
+	 * FlatMapFunction to determine the language of tweets if possible
+	 */
+	public static class SelectLanguageFlatMap extends
+			JSONParseFlatMap<String, String> {
+
+		private static final long serialVersionUID = 1L;
+
+		/**
+		 * Select the language from the incoming JSON text
+		 */
+		@Override
+		public void flatMap(String value, Collector<String> out) throws Exception {
+			try{
+				out.collect(getString(value, "lang"));
+			}
+			catch (JSONException e){
+				out.collect("");
+			}
+		}
+
+	}
+
+	public static void main(String[] args) throws Exception {
+
+		String path = new String();
+
+		if (args != null && args.length == 1) {
+			path = args[0];
+		} else {
+			System.err.println("USAGE:\nTwitterLocal <pathToPropertiesFile>");
+			return;
+		}
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStream<String> streamSource = env.addSource(new TwitterSource(path, NUMBEROFTWEETS));
+
+
+		DataStream<Tuple2<String, Integer>> dataStream = streamSource
+				.flatMap(new SelectLanguageFlatMap())
+				.map(new MapFunction<String, Tuple2<String, Integer>>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public Tuple2<String, Integer> map(String value) throws Exception {
+						return new Tuple2<String, Integer>(value, 1);
+					}
+				})
+				.keyBy(0)
+				.sum(1);
+
+		dataStream.print();
+
+		env.execute();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties b/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties
new file mode 100644
index 0000000..1ca4143
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties
@@ -0,0 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+secret=***
+consumerSecret=***
+token=***-***
+consumerKey=***

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java b/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
new file mode 100644
index 0000000..b1d4115
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.json;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.flink.streaming.connectors.json.JSONParser;
+import org.apache.sling.commons.json.JSONException;
+import org.apache.sling.commons.json.JSONObject;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(Parameterized.class)
+public class JSONParserTest {
+
+	private String jsonText;
+	private String searchedField;
+
+	public JSONParserTest(String text, String field) {
+		jsonText = text;
+		searchedField = field;
+	}
+
+	@Parameters
+	public static Collection<Object[]> initParameterList() {
+
+		Object[][] parameterList = new Object[][] { 
+				{ "{\"key\":\"value\"}", 							"key" },
+				{ "{\"key\":[\"value\"]}", 							"key[0]" },
+				{ "{\"key\":[{\"key\":\"value\"}]}", 				"key[0].key" },
+				{ "{\"key\":[{\"key\":[{\"key\":\"value\"}]}]}", 	"key[0].key[0].key"},
+				{ "{\"key\":[1,[{\"key\":\"value\"}]]}", 			"key[1][0].key" },
+				{ "{\"key\":[1,[[\"key\",2,\"value\"]]]}", 			"key[1][0][2]" },
+				{ "{\"key\":{\"key\":{\"otherKey\":\"wrongValue\",\"key\":\"value\"},\"otherKey\":\"wrongValue\"},\"otherKey\":\"wrongValue\"}" , "key.key.key"}
+				};
+
+		return Arrays.asList(parameterList);
+	}
+
+	@Test
+	public void test() {
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+			String expected = "{\"retValue\":\"value\"}";
+
+			assertTrue(expected.equals(jo.toString()));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java b/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
new file mode 100644
index 0000000..8851086
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.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.connectors.json;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.streaming.connectors.json.JSONParser;
+import org.apache.sling.commons.json.JSONException;
+import org.apache.sling.commons.json.JSONObject;
+import org.junit.Test;
+
+
+public class JSONParserTest2 {
+	
+	@Test
+	public void testGetBooleanFunction() {
+		String jsonText = "{\"key\":true}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertTrue(jo.getBoolean("retValue"));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+	
+	@Test
+	public void testGetDoubleFunction() {
+		double expected = 12345.12345;
+		String jsonText = "{\"key\":" + expected + "}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertEquals(expected,jo.getDouble("retValue"),0.000001);
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+	
+	@Test
+	public void testGetIntFunction() {
+		int expected = 15;
+		String jsonText = "{\"key\":" + expected + "}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertEquals(expected,jo.getInt("retValue"));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+
+	@Test
+	public void testGetLongFunction() {
+		long expected = 111111111111L;
+		String jsonText = "{\"key\":" + expected + "}";
+		String searchedField = "key";
+		try {
+			JSONParser parser = new JSONParser(jsonText);
+			JSONObject jo = parser.parse(searchedField);
+
+			assertEquals(expected,jo.getLong("retValue"));
+		} 
+		catch (JSONException e) {
+			fail();
+		}
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..9ede613
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties
@@ -0,0 +1,27 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+log4j.rootLogger=INFO, testlogger
+
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..45b3b92
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml
@@ -0,0 +1,30 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+    <logger name="org.apache.flink.streaming" level="WARN"/>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/pom.xml b/flink-streaming-connectors/pom.xml
new file mode 100644
index 0000000..dc395b2
--- /dev/null
+++ b/flink-streaming-connectors/pom.xml
@@ -0,0 +1,66 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-streaming-connectors-parent</artifactId>
+	<name>flink-streaming-connectors</name>
+
+	<packaging>pom</packaging>
+
+	<modules>
+		<module>flink-connector-flume</module>
+		<module>flink-connector-kafka</module>
+		<module>flink-connector-elasticsearch</module>
+		<module>flink-connector-rabbitmq</module>
+		<module>flink-connector-twitter</module>
+		<module>flink-connector-nifi</module>
+	</modules>
+
+	<!-- See main pom.xml for explanation of profiles -->
+	<profiles>
+		<profile>
+			<id>hadoop-2</id>
+			<activation>
+				<property>
+					<!-- Please do not remove the 'hadoop2' comment. See ./tools/generate_specific_pom.sh -->
+					<!--hadoop2--><name>!hadoop.profile</name>
+				</property>
+			</activation>
+			<modules>
+				<!-- Include the flink-fs-tests project only for HD2.
+				 	The HDFS minicluster interfaces changed between the two versions.
+				 -->
+				<module>flink-connector-filesystem</module>
+			</modules>
+		</profile>
+	</profiles>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/pom.xml b/flink-streaming-examples/pom.xml
new file mode 100644
index 0000000..5e06411
--- /dev/null
+++ b/flink-streaming-examples/pom.xml
@@ -0,0 +1,535 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-streaming-examples</artifactId>
+	<name>flink-streaming-examples</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-scala</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java-examples</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-twitter</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<!-- get default data from flink-java-examples package -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-dependency-plugin</artifactId>
+				<version>2.9</version><!--$NO-MVN-MAN-VER$-->
+				<executions>
+					<execution>
+						<id>unpack</id>
+						<phase>prepare-package</phase>
+						<goals>
+							<goal>unpack</goal>
+						</goals>
+						<configuration>
+							<artifactItems>
+								<!-- For WordCount example data -->
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-java-examples</artifactId>
+									<version>${project.version}</version>
+									<type>jar</type>
+									<overWrite>false</overWrite>
+									<outputDirectory>${project.build.directory}/classes</outputDirectory>
+									<includes>org/apache/flink/examples/java/wordcount/util/WordCountData.class</includes>
+								</artifactItem>
+								<!-- For JSON utilities -->
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-connector-twitter</artifactId>
+									<version>${project.version}</version>
+									<type>jar</type>
+									<overWrite>false</overWrite>
+									<outputDirectory>${project.build.directory}/classes</outputDirectory>
+									<includes>org/apache/flink/streaming/connectors/json/*</includes>
+								</artifactItem>
+							</artifactItems>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+
+			<!-- self-contained jars for each example -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<version>2.4</version><!--$NO-MVN-MAN-VER$-->
+				<executions>
+					<!-- Default Execution -->
+					<execution>
+						<id>default</id>
+						<phase>package</phase>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+					
+					<!-- Iteration -->
+					<execution>
+						<id>Iteration</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>Iteration</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.iteration.IterateExample</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/iteration/*.class</include>			
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- IncrementalLearning -->
+					<execution>
+						<id>IncrementalLearning</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>IncrementalLearning</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.ml.IncrementalLearningSkeleton</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/ml/*.class</include>			
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- Twitter -->
+					<execution>
+						<id>Twitter</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>Twitter</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.twitter.TwitterStream</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/twitter/*.class</include>
+								<include>org/apache/flink/streaming/examples/twitter/util/*.class</include>
+								<include>org/apache/flink/streaming/connectors/json/*.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- WindowJoin -->
+					<execution>
+						<id>WindowJoin</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>WindowJoin</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.join.WindowJoin</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/join/*.class</include>			
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- WordCountPOJO -->
+					<execution>
+						<id>WordCountPOJO</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>WordCountPOJO</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.wordcount.PojoExample</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/wordcount/PojoExample.class</include>
+								<include>org/apache/flink/streaming/examples/wordcount/PojoExample$*.class</include>
+								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>			
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- WordCount -->
+					<execution>
+						<id>WordCount</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>WordCount</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.wordcount.WordCount</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/wordcount/WordCount.class</include>
+								<include>org/apache/flink/streaming/examples/wordcount/WordCount$*.class</include>
+								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>				
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- WindowWordCount -->
+					<execution>
+						<id>WindowWordCount</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>WindowWordCount</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.windowing.WindowWordCount</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/windowing/WindowWordCount.class</include>
+								<include>org/apache/flink/streaming/examples/wordcount/WordCount.class</include>
+								<include>org/apache/flink/streaming/examples/wordcount/WordCount$*.class</include>
+								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- SocketTextStreamWordCount -->
+					<execution>
+						<id>SocketTextStreamWordCount</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>SocketTextStreamWordCount</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.socket.SocketTextStreamWordCount</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.class</include>
+								<include>org/apache/flink/streaming/examples/wordcount/WordCount.class</include>
+								<include>org/apache/flink/streaming/examples/wordcount/WordCount$*.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- TopSpeedWindowing -->
+					<execution>
+						<id>TopSpeedWindowing</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>TopSpeedWindowing</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.windowing.TopSpeedWindowing</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class</include>
+								<include>org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- SessionWindowing -->
+					<execution>
+						<id>SessionWindowing</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<classifier>SessionWindowing</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.streaming.examples.windowing.SessionWindowing</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<include>org/apache/flink/streaming/examples/windowing/SessionWindowing.class</include>
+								<include>org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
+				</executions>
+			</plugin>
+
+
+			<!-- Scala Compiler -->
+			<plugin>
+				<groupId>net.alchim31.maven</groupId>
+				<artifactId>scala-maven-plugin</artifactId>
+				<version>3.1.4</version>
+				<executions>
+					<!-- Run scala compiler in the process-resources phase, so that dependencies on
+						scala classes can be resolved later in the (Java) compile phase -->
+					<execution>
+						<id>scala-compile-first</id>
+						<phase>process-resources</phase>
+						<goals>
+							<goal>compile</goal>
+						</goals>
+					</execution>
+ 
+					<!-- Run scala compiler in the process-test-resources phase, so that dependencies on
+						 scala classes can be resolved later in the (Java) test-compile phase -->
+					<execution>
+						<id>scala-test-compile</id>
+						<phase>process-test-resources</phase>
+						<goals>
+							<goal>testCompile</goal>
+						</goals>
+					</execution>
+				</executions>
+				<configuration>
+					<jvmArgs>
+						<jvmArg>-Xms128m</jvmArg>
+						<jvmArg>-Xmx512m</jvmArg>
+					</jvmArgs>
+				</configuration>
+			</plugin>
+			
+			<!-- Eclipse Integration -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-eclipse-plugin</artifactId>
+				<version>2.8</version>
+				<configuration>
+					<downloadSources>true</downloadSources>
+					<projectnatures>
+						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
+						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
+					</projectnatures>
+					<buildcommands>
+						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
+					</buildcommands>
+					<classpathContainers>
+						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
+						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
+					</classpathContainers>
+					<excludes>
+						<exclude>org.scala-lang:scala-library</exclude>
+						<exclude>org.scala-lang:scala-compiler</exclude>
+					</excludes>
+					<sourceIncludes>
+						<sourceInclude>**/*.scala</sourceInclude>
+						<sourceInclude>**/*.java</sourceInclude>
+					</sourceIncludes>
+				</configuration>
+			</plugin>
+
+			<!-- Adding scala source directories to build path -->
+			<plugin>
+				<groupId>org.codehaus.mojo</groupId>
+				<artifactId>build-helper-maven-plugin</artifactId>
+				<version>1.7</version>
+				<executions>
+					<!-- Add src/main/scala to eclipse build path -->
+					<execution>
+						<id>add-source</id>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>add-source</goal>
+						</goals>
+						<configuration>
+							<sources>
+								<source>src/main/scala</source>
+							</sources>
+						</configuration>
+					</execution>
+					<!-- Add src/test/scala to eclipse build path -->
+					<execution>
+						<id>add-test-source</id>
+						<phase>generate-test-sources</phase>
+						<goals>
+							<goal>add-test-source</goal>
+						</goals>
+						<configuration>
+							<sources>
+								<source>src/test/scala</source>
+							</sources>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+
+			<plugin>
+				<groupId>org.scalastyle</groupId>
+				<artifactId>scalastyle-maven-plugin</artifactId>
+				<version>0.5.0</version>
+				<executions>
+					<execution>
+						<goals>
+							<goal>check</goal>
+						</goals>
+					</execution>
+				</executions>
+				<configuration>
+					<verbose>false</verbose>
+					<failOnViolation>true</failOnViolation>
+					<includeTestSourceDirectory>true</includeTestSourceDirectory>
+					<failOnWarning>false</failOnWarning>
+					<sourceDirectory>${basedir}/src/main/scala</sourceDirectory>
+					<testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory>
+					<configLocation>${project.basedir}/../tools/maven/scalastyle-config.xml</configLocation>
+					<outputFile>${project.basedir}/scalastyle-output.xml</outputFile>
+					<outputEncoding>UTF-8</outputEncoding>
+				</configuration>
+			</plugin>
+		</plugins>
+		
+		<pluginManagement>
+			<plugins>
+				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+				<plugin>
+					<groupId>org.eclipse.m2e</groupId>
+					<artifactId>lifecycle-mapping</artifactId>
+					<version>1.0.0</version>
+					<configuration>
+						<lifecycleMappingMetadata>
+							<pluginExecutions>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-dependency-plugin</artifactId>
+										<versionRange>[2.9,)</versionRange>
+										<goals>
+											<goal>unpack</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+							</pluginExecutions>
+						</lifecycleMappingMetadata>
+					</configuration>
+				</plugin>
+			</plugins>
+		</pluginManagement>
+
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
new file mode 100644
index 0000000..2cf66b9
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
@@ -0,0 +1,247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.iteration;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.IterativeStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Example illustrating iterations in Flink streaming. <p/> <p> The program sums up random numbers and counts additions
+ * it performs to reach a specific threshold in an iterative streaming fashion. </p>
+ * <p/>
+ * <p/>
+ * This example shows how to use: <ul> <li>streaming iterations, <li>buffer timeout to enhance latency, <li>directed
+ * outputs. </ul>
+ */
+public class IterateExample {
+
+	private static final int BOUND = 100;
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up input for the stream of integer pairs
+
+		// obtain execution environment and set setBufferTimeout to 1 to enable
+		// continuous flushing of the output buffers (lowest latency)
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment()
+				.setBufferTimeout(1);
+
+		// create input stream of integer pairs
+		DataStream<Tuple2<Integer, Integer>> inputStream;
+		if (fileInput) {
+			inputStream = env.readTextFile(inputPath).map(new FibonacciInputMap());
+		} else {
+			inputStream = env.addSource(new RandomFibonacciSource());
+		}
+
+		// create an iterative data stream from the input with 5 second timeout
+		IterativeStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> it = inputStream.map(new InputMap())
+				.iterate(5000);
+
+		// apply the step function to get the next Fibonacci number
+		// increment the counter and split the output with the output selector
+		SplitStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> step = it.map(new Step())
+				.split(new MySelector());
+
+		// close the iteration by selecting the tuples that were directed to the
+		// 'iterate' channel in the output selector
+		it.closeWith(step.select("iterate"));
+
+		// to produce the final output select the tuples directed to the
+		// 'output' channel then get the input pairs that have the greatest iteration counter
+		// on a 1 second sliding window
+		DataStream<Tuple2<Tuple2<Integer, Integer>, Integer>> numbers = step.select("output")
+				.map(new OutputMap());
+
+		// emit results
+		if (fileOutput) {
+			numbers.writeAsText(outputPath, 1);
+		} else {
+			numbers.print();
+		}
+
+		// execute the program
+		env.execute("Streaming Iteration Example");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Generate BOUND number of random integer pairs from the range from 0 to BOUND/2
+	 */
+	private static class RandomFibonacciSource implements SourceFunction<Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		private Random rnd = new Random();
+
+		private volatile boolean isRunning = true;
+		private int counter = 0;
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
+
+			while (isRunning && counter < BOUND) {
+				int first = rnd.nextInt(BOUND / 2 - 1) + 1;
+				int second = rnd.nextInt(BOUND / 2 - 1) + 1;
+
+				ctx.collect(new Tuple2<Integer, Integer>(first, second));
+				counter++;
+				Thread.sleep(50L);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+	}
+
+	/**
+	 * Generate random integer pairs from the range from 0 to BOUND/2
+	 */
+	private static class FibonacciInputMap implements MapFunction<String, Tuple2<Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Integer, Integer> map(String value) throws Exception {
+			String record = value.substring(1, value.length() - 1);
+			String[] splitted = record.split(",");
+			return new Tuple2<Integer, Integer>(Integer.parseInt(splitted[0]), Integer.parseInt(splitted[1]));
+		}
+	}
+
+	/**
+	 * Map the inputs so that the next Fibonacci numbers can be calculated while preserving the original input tuple A
+	 * counter is attached to the tuple and incremented in every iteration step
+	 */
+	public static class InputMap implements MapFunction<Tuple2<Integer, Integer>, Tuple5<Integer, Integer, Integer,
+			Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple5<Integer, Integer, Integer, Integer, Integer> map(Tuple2<Integer, Integer> value) throws
+				Exception {
+			return new Tuple5<Integer, Integer, Integer, Integer, Integer>(value.f0, value.f1, value.f0, value.f1, 0);
+		}
+	}
+
+	/**
+	 * Iteration step function that calculates the next Fibonacci number
+	 */
+	public static class Step implements
+			MapFunction<Tuple5<Integer, Integer, Integer, Integer, Integer>, Tuple5<Integer, Integer, Integer,
+					Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple5<Integer, Integer, Integer, Integer, Integer> map(Tuple5<Integer, Integer, Integer, Integer,
+				Integer> value) throws Exception {
+			return new Tuple5<Integer, Integer, Integer, Integer, Integer>(value.f0, value.f1, value.f3, value.f2 +
+					value.f3, ++value.f4);
+		}
+	}
+
+	/**
+	 * OutputSelector testing which tuple needs to be iterated again.
+	 */
+	public static class MySelector implements OutputSelector<Tuple5<Integer, Integer, Integer, Integer, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Iterable<String> select(Tuple5<Integer, Integer, Integer, Integer, Integer> value) {
+			List<String> output = new ArrayList<String>();
+			if (value.f2 < BOUND && value.f3 < BOUND) {
+				output.add("iterate");
+			} else {
+				output.add("output");
+			}
+			return output;
+		}
+	}
+
+	/**
+	 * Giving back the input pair and the counter
+	 */
+	public static class OutputMap implements MapFunction<Tuple5<Integer, Integer, Integer, Integer, Integer>,
+			Tuple2<Tuple2<Integer, Integer>, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<Tuple2<Integer, Integer>, Integer> map(Tuple5<Integer, Integer, Integer, Integer, Integer>
+				value) throws
+				Exception {
+			return new Tuple2<Tuple2<Integer, Integer>, Integer>(new Tuple2<Integer, Integer>(value.f0, value.f1),
+					value.f4);
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileInput = false;
+	private static boolean fileOutput = false;
+	private static String inputPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			if (args.length == 1) {
+				fileOutput = true;
+				outputPath = args[0];
+			} else if (args.length == 2) {
+				fileInput = true;
+				inputPath = args[0];
+				fileOutput = true;
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: IterateExample <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing IterateExample with generated data.");
+			System.out.println("  Provide parameter to write to file.");
+			System.out.println("  Usage: IterateExample <result path>");
+		}
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java
new file mode 100644
index 0000000..0077459
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.iteration.util;
+
+public class IterateExampleData {
+	public static final String INPUT_PAIRS = "(1,40)\n" + "(29,38)\n" + "(11,15)\n" + "(17,39)\n" + "(24,41)\n" +
+			"(7,33)\n" + "(20,2)\n" + "(11,5)\n" + "(3,16)\n" + "(23,36)\n" + "(15,23)\n" + "(28,13)\n" + "(1,1)\n" +
+			"(10,6)\n" + "(21,5)\n" + "(14,36)\n" + "(17,15)\n" + "(7,9)";
+
+	public static final String RESULTS = "((1,40),3)\n" + "((24,41),2)\n" + "((3,16),5)\n" + "((1,1),10)\n" +
+			"((17,15),4)\n" + "((29,38),2)\n" + "((7,33),3)\n" + "((23,36),3)\n" + "((10,6),6)\n" + "((7,9),5)\n" +
+			"((11,15),4)\n" + "((20,2),5)\n" + "((15,23),4)\n" + "((21,5),5)\n" +
+			"((17,39),3)\n" + "((11,5),6)\n" + "((28,13),4)\n" + "((14,36),3)";
+
+	private IterateExampleData() {
+	}
+}


[36/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
deleted file mode 100644
index 45cfff1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ /dev/null
@@ -1,444 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobgraph.ScheduleMode;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
-import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.runtime.tasks.StreamIterationHead;
-import org.apache.flink.streaming.runtime.tasks.StreamIterationTail;
-import org.apache.flink.util.InstantiationUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StreamingJobGraphGenerator {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamingJobGraphGenerator.class);
-
-	private StreamGraph streamGraph;
-
-	private Map<Integer, JobVertex> jobVertices;
-	private JobGraph jobGraph;
-	private Collection<Integer> builtVertices;
-
-	private List<StreamEdge> physicalEdgesInOrder;
-
-	private Map<Integer, Map<Integer, StreamConfig>> chainedConfigs;
-
-	private Map<Integer, StreamConfig> vertexConfigs;
-	private Map<Integer, String> chainedNames;
-
-	public StreamingJobGraphGenerator(StreamGraph streamGraph) {
-		this.streamGraph = streamGraph;
-	}
-
-	private void init() {
-		this.jobVertices = new HashMap<Integer, JobVertex>();
-		this.builtVertices = new HashSet<Integer>();
-		this.chainedConfigs = new HashMap<Integer, Map<Integer, StreamConfig>>();
-		this.vertexConfigs = new HashMap<Integer, StreamConfig>();
-		this.chainedNames = new HashMap<Integer, String>();
-		this.physicalEdgesInOrder = new ArrayList<StreamEdge>();
-	}
-
-	public JobGraph createJobGraph(String jobName) {
-		jobGraph = new JobGraph(jobName);
-
-		// make sure that all vertices start immediately
-		jobGraph.setScheduleMode(ScheduleMode.ALL);
-
-		init();
-
-		setChaining();
-
-		setPhysicalEdges();
-
-		setSlotSharing();
-		
-		configureCheckpointing();
-
-		configureExecutionRetries();
-		
-		configureExecutionRetryDelay();
-
-		try {
-			InstantiationUtil.writeObjectToConfig(this.streamGraph.getExecutionConfig(), this.jobGraph.getJobConfiguration(), ExecutionConfig.CONFIG_KEY);
-		} catch (IOException e) {
-			throw new RuntimeException("Config object could not be written to Job Configuration: ", e);
-		}
-		
-		return jobGraph;
-	}
-
-	private void setPhysicalEdges() {
-		Map<Integer, List<StreamEdge>> physicalInEdgesInOrder = new HashMap<Integer, List<StreamEdge>>();
-
-		for (StreamEdge edge : physicalEdgesInOrder) {
-			int target = edge.getTargetId();
-
-			List<StreamEdge> inEdges = physicalInEdgesInOrder.get(target);
-
-			// create if not set
-			if (inEdges == null) {
-				inEdges = new ArrayList<StreamEdge>();
-				physicalInEdgesInOrder.put(target, inEdges);
-			}
-
-			inEdges.add(edge);
-		}
-
-		for (Map.Entry<Integer, List<StreamEdge>> inEdges : physicalInEdgesInOrder.entrySet()) {
-			int vertex = inEdges.getKey();
-			List<StreamEdge> edgeList = inEdges.getValue();
-
-			vertexConfigs.get(vertex).setInPhysicalEdges(edgeList);
-		}
-	}
-
-	private void setChaining() {
-		for (Integer sourceName : streamGraph.getSourceIDs()) {
-			createChain(sourceName, sourceName);
-		}
-	}
-
-	private List<StreamEdge> createChain(Integer startNode, Integer current) {
-
-		if (!builtVertices.contains(startNode)) {
-
-			List<StreamEdge> transitiveOutEdges = new ArrayList<StreamEdge>();
-
-			List<StreamEdge> chainableOutputs = new ArrayList<StreamEdge>();
-			List<StreamEdge> nonChainableOutputs = new ArrayList<StreamEdge>();
-
-			for (StreamEdge outEdge : streamGraph.getStreamNode(current).getOutEdges()) {
-				if (isChainable(outEdge)) {
-					chainableOutputs.add(outEdge);
-				} else {
-					nonChainableOutputs.add(outEdge);
-				}
-			}
-
-			for (StreamEdge chainable : chainableOutputs) {
-				transitiveOutEdges.addAll(createChain(startNode, chainable.getTargetId()));
-			}
-
-			for (StreamEdge nonChainable : nonChainableOutputs) {
-				transitiveOutEdges.add(nonChainable);
-				createChain(nonChainable.getTargetId(), nonChainable.getTargetId());
-			}
-
-			chainedNames.put(current, createChainedName(current, chainableOutputs));
-
-			StreamConfig config = current.equals(startNode) ? createProcessingVertex(startNode)
-					: new StreamConfig(new Configuration());
-
-			setVertexConfig(current, config, chainableOutputs, nonChainableOutputs);
-
-			if (current.equals(startNode)) {
-
-				config.setChainStart();
-				config.setOutEdgesInOrder(transitiveOutEdges);
-				config.setOutEdges(streamGraph.getStreamNode(current).getOutEdges());
-
-				for (StreamEdge edge : transitiveOutEdges) {
-					connect(startNode, edge);
-				}
-
-				config.setTransitiveChainedTaskConfigs(chainedConfigs.get(startNode));
-
-			} else {
-
-				Map<Integer, StreamConfig> chainedConfs = chainedConfigs.get(startNode);
-
-				if (chainedConfs == null) {
-					chainedConfigs.put(startNode, new HashMap<Integer, StreamConfig>());
-				}
-				chainedConfigs.get(startNode).put(current, config);
-			}
-
-			return transitiveOutEdges;
-
-		} else {
-			return new ArrayList<StreamEdge>();
-		}
-	}
-
-	private String createChainedName(Integer vertexID, List<StreamEdge> chainedOutputs) {
-		String operatorName = streamGraph.getStreamNode(vertexID).getOperatorName();
-		if (chainedOutputs.size() > 1) {
-			List<String> outputChainedNames = new ArrayList<String>();
-			for (StreamEdge chainable : chainedOutputs) {
-				outputChainedNames.add(chainedNames.get(chainable.getTargetId()));
-			}
-			return operatorName + " -> (" + StringUtils.join(outputChainedNames, ", ") + ")";
-		} else if (chainedOutputs.size() == 1) {
-			return operatorName + " -> " + chainedNames.get(chainedOutputs.get(0).getTargetId());
-		} else {
-			return operatorName;
-		}
-
-	}
-
-	private StreamConfig createProcessingVertex(Integer vertexID) {
-
-		JobVertex jobVertex = new JobVertex(chainedNames.get(vertexID));
-		StreamNode vertex = streamGraph.getStreamNode(vertexID);
-
-		jobVertex.setInvokableClass(vertex.getJobVertexClass());
-
-		int parallelism = vertex.getParallelism();
-
-		if (parallelism > 0) {
-			jobVertex.setParallelism(parallelism);
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Parallelism set: {} for {}", parallelism, vertexID);
-		}
-
-		if (vertex.getInputFormat() != null) {
-			jobVertex.setInputSplitSource(vertex.getInputFormat());
-		}
-
-		jobVertices.put(vertexID, jobVertex);
-		builtVertices.add(vertexID);
-		jobGraph.addVertex(jobVertex);
-
-		return new StreamConfig(jobVertex.getConfiguration());
-	}
-
-	@SuppressWarnings("unchecked")
-	private void setVertexConfig(Integer vertexID, StreamConfig config,
-			List<StreamEdge> chainableOutputs, List<StreamEdge> nonChainableOutputs) {
-
-		StreamNode vertex = streamGraph.getStreamNode(vertexID);
-
-		config.setVertexID(vertexID);
-		config.setBufferTimeout(vertex.getBufferTimeout());
-
-		config.setTypeSerializerIn1(vertex.getTypeSerializerIn1());
-		config.setTypeSerializerIn2(vertex.getTypeSerializerIn2());
-		config.setTypeSerializerOut(vertex.getTypeSerializerOut());
-
-		config.setStreamOperator(vertex.getOperator());
-		config.setOutputSelectorWrapper(vertex.getOutputSelectorWrapper());
-
-		config.setNumberOfOutputs(nonChainableOutputs.size());
-		config.setNonChainedOutputs(nonChainableOutputs);
-		config.setChainedOutputs(chainableOutputs);
-
-		config.setCheckpointingEnabled(streamGraph.isCheckpointingEnabled());
-		if (streamGraph.isCheckpointingEnabled()) {
-			config.setCheckpointMode(streamGraph.getCheckpointingMode());
-			config.setStateBackend(streamGraph.getStateBackend());
-		} else {
-			// the at least once input handler is slightly cheaper (in the absence of checkpoints),
-			// so we use that one if checkpointing is not enabled
-			config.setCheckpointMode(CheckpointingMode.AT_LEAST_ONCE);
-		}
-		config.setStatePartitioner((KeySelector<?, Serializable>) vertex.getStatePartitioner());
-		config.setStateKeySerializer(vertex.getStateKeySerializer());
-
-		
-		Class<? extends AbstractInvokable> vertexClass = vertex.getJobVertexClass();
-
-		if (vertexClass.equals(StreamIterationHead.class)
-				|| vertexClass.equals(StreamIterationTail.class)) {
-			config.setIterationId(streamGraph.getBrokerID(vertexID));
-			config.setIterationWaitTime(streamGraph.getLoopTimeout(vertexID));
-		}
-
-		List<StreamEdge> allOutputs = new ArrayList<StreamEdge>(chainableOutputs);
-		allOutputs.addAll(nonChainableOutputs);
-
-		vertexConfigs.put(vertexID, config);
-	}
-
-	private void connect(Integer headOfChain, StreamEdge edge) {
-
-		physicalEdgesInOrder.add(edge);
-
-		Integer downStreamvertexID = edge.getTargetId();
-
-		JobVertex headVertex = jobVertices.get(headOfChain);
-		JobVertex downStreamVertex = jobVertices.get(downStreamvertexID);
-
-		StreamConfig downStreamConfig = new StreamConfig(downStreamVertex.getConfiguration());
-
-		downStreamConfig.setNumberOfInputs(downStreamConfig.getNumberOfInputs() + 1);
-
-		StreamPartitioner<?> partitioner = edge.getPartitioner();
-		if (partitioner instanceof ForwardPartitioner) {
-			downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE);
-		} else {
-			downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.ALL_TO_ALL);
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("CONNECTED: {} - {} -> {}", partitioner.getClass().getSimpleName(),
-					headOfChain, downStreamvertexID);
-		}
-	}
-
-	private boolean isChainable(StreamEdge edge) {
-		StreamNode upStreamVertex = edge.getSourceVertex();
-		StreamNode downStreamVertex = edge.getTargetVertex();
-
-		StreamOperator<?> headOperator = upStreamVertex.getOperator();
-		StreamOperator<?> outOperator = downStreamVertex.getOperator();
-
-		return downStreamVertex.getInEdges().size() == 1
-				&& outOperator != null
-				&& headOperator != null
-				&& upStreamVertex.getSlotSharingID() == downStreamVertex.getSlotSharingID()
-				&& upStreamVertex.getSlotSharingID() != -1
-				&& (outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS ||
-					outOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS)
-				&& (headOperator.getChainingStrategy() == ChainingStrategy.HEAD ||
-					headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS ||
-					headOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS)
-				&& (edge.getPartitioner() instanceof ForwardPartitioner || downStreamVertex
-						.getParallelism() == 1)
-				&& upStreamVertex.getParallelism() == downStreamVertex.getParallelism()
-				&& (streamGraph.isChainingEnabled() ||
-					outOperator.getChainingStrategy() == ChainingStrategy.FORCE_ALWAYS);
-	}
-
-	private void setSlotSharing() {
-
-		Map<Integer, SlotSharingGroup> slotSharingGroups = new HashMap<Integer, SlotSharingGroup>();
-
-		for (Entry<Integer, JobVertex> entry : jobVertices.entrySet()) {
-
-			int slotSharingID = streamGraph.getStreamNode(entry.getKey()).getSlotSharingID();
-
-			if (slotSharingID != -1) {
-				SlotSharingGroup group = slotSharingGroups.get(slotSharingID);
-				if (group == null) {
-					group = new SlotSharingGroup();
-					slotSharingGroups.put(slotSharingID, group);
-				}
-				entry.getValue().setSlotSharingGroup(group);
-			}
-		}
-
-		for (Tuple2<StreamNode, StreamNode> pair : streamGraph.getIterationSourceSinkPairs()) {
-
-			CoLocationGroup ccg = new CoLocationGroup();
-
-			JobVertex source = jobVertices.get(pair.f0.getId());
-			JobVertex sink = jobVertices.get(pair.f1.getId());
-
-			ccg.addVertex(source);
-			ccg.addVertex(sink);
-			source.updateCoLocationGroup(ccg);
-			sink.updateCoLocationGroup(ccg);
-		}
-
-	}
-	
-	private void configureCheckpointing() {
-		if (streamGraph.isCheckpointingEnabled()) {
-			long interval = streamGraph.getCheckpointingInterval();
-			if (interval < 1) {
-				throw new IllegalArgumentException("The checkpoint interval must be positive");
-			}
-
-			// collect the vertices that receive "trigger checkpoint" messages.
-			// currently, these are all the sources
-			List<JobVertexID> triggerVertices = new ArrayList<JobVertexID>();
-
-			// collect the vertices that need to acknowledge the checkpoint
-			// currently, these are all vertices
-			List<JobVertexID> ackVertices = new ArrayList<JobVertexID>(jobVertices.size());
-
-			// collect the vertices that receive "commit checkpoint" messages
-			// currently, these are all certices
-			List<JobVertexID> commitVertices = new ArrayList<JobVertexID>();
-			
-			
-			for (JobVertex vertex : jobVertices.values()) {
-				if (vertex.isInputVertex()) {
-					triggerVertices.add(vertex.getID());
-				}
-				// TODO: add check whether the user function implements the checkpointing interface
-				commitVertices.add(vertex.getID());
-				ackVertices.add(vertex.getID());
-			}
-
-			JobSnapshottingSettings settings = new JobSnapshottingSettings(
-					triggerVertices, ackVertices, commitVertices, interval);
-			jobGraph.setSnapshotSettings(settings);
-
-			// if the user enabled checkpointing, the default number of exec retries is infinitive.
-			int executionRetries = streamGraph.getExecutionConfig().getNumberOfExecutionRetries();
-			if(executionRetries == -1) {
-				streamGraph.getExecutionConfig().setNumberOfExecutionRetries(Integer.MAX_VALUE);
-			}
-			long executionRetryDelay = streamGraph.getExecutionConfig().getExecutionRetryDelay();
-			if(executionRetryDelay == -1) {
-				streamGraph.getExecutionConfig().setExecutionRetryDelay(100 * 1000);
-			}
-		}
-	}
-
-	private void configureExecutionRetries() {
-		int executionRetries = streamGraph.getExecutionConfig().getNumberOfExecutionRetries();
-		if (executionRetries != -1) {
-			jobGraph.setNumberOfExecutionRetries(executionRetries);
-		} else {
-			// if the user didn't configure anything, the number of retries is 0.
-			jobGraph.setNumberOfExecutionRetries(0);
-		}
-	}
-	
-	private void configureExecutionRetryDelay() {
-		long executionRetryDelay = streamGraph.getExecutionConfig().getExecutionRetryDelay();
-		if (executionRetryDelay != -1) {
-			jobGraph.setExecutionRetryDelay(executionRetryDelay);
-		} else {
-			jobGraph.setExecutionRetryDelay(100 * 1000);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
deleted file mode 100644
index 078679d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ /dev/null
@@ -1,373 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.runtime.state.KvState;
-import org.apache.flink.runtime.state.KvStateSnapshot;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Base class for all stream operators. Operators that contain a user function should extend the class 
- * {@link AbstractUdfStreamOperator} instead (which is a specialized subclass of this class). 
- * 
- * <p>For concrete implementations, one of the following two interfaces must also be implemented, to
- * mark the operator as unary or binary:
- * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator} or
- * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator}.
- *
- * <p>Methods of {@code StreamOperator} are guaranteed not to be called concurrently. Also, if using
- * the timer service, timer callbacks are also guaranteed not to be called concurrently with
- * methods on {@code StreamOperator}.
- *
- * @param <OUT> The output type of the operator
- */
-public abstract class AbstractStreamOperator<OUT> 
-		implements StreamOperator<OUT>, java.io.Serializable {
-
-	private static final long serialVersionUID = 1L;
-	
-	/** The logger used by the operator class and its subclasses */
-	protected static final Logger LOG = LoggerFactory.getLogger(AbstractStreamOperator.class);
-
-	// ----------- configuration properties -------------
-
-	// A sane default for most operators
-	protected ChainingStrategy chainingStrategy = ChainingStrategy.HEAD;
-	
-	private boolean inputCopyDisabled = false;
-	
-	// ---------------- runtime fields ------------------
-
-	/** The task that contains this operator (and other operators in the same chain) */
-	private transient StreamTask<?, ?> container;
-	
-	private transient StreamConfig config;
-
-	protected transient Output<StreamRecord<OUT>> output;
-
-	/** The runtime context for UDFs */
-	private transient StreamingRuntimeContext runtimeContext;
-
-	
-	// ---------------- key/value state ------------------
-	
-	/** key selector used to get the key for the state. Non-null only is the operator uses key/value state */
-	private transient KeySelector<?, ?> stateKeySelector;
-	
-	private transient KvState<?, ?, ?>[] keyValueStates;
-	
-	private transient HashMap<String, KvState<?, ?, ?>> keyValueStatesByName;
-	
-	private transient TypeSerializer<?> keySerializer;
-	
-	private transient HashMap<String, KvStateSnapshot<?, ?, ?>> keyValueStateSnapshots;
-	
-	// ------------------------------------------------------------------------
-	//  Life Cycle
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output) {
-		this.container = containingTask;
-		this.config = config;
-		this.output = output;
-		this.runtimeContext = new StreamingRuntimeContext(this, container.getEnvironment(), container.getAccumulatorMap());
-	}
-
-	/**
-	 * This method is called immediately before any elements are processed, it should contain the
-	 * operator's initialization logic.
-	 *
-	 * <p>The default implementation does nothing.
-	 * 
-	 * @throws Exception An exception in this method causes the operator to fail.
-	 */
-	@Override
-	public void open() throws Exception {}
-
-	/**
-	 * This method is called after all records have been added to the operators via the methods
-	 * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator#processElement(StreamRecord)}, or
-	 * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator#processElement1(StreamRecord)} and
-	 * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator#processElement2(StreamRecord)}.
-
-	 * <p>The method is expected to flush all remaining buffered data. Exceptions during this flushing
-	 * of buffered should be propagated, in order to cause the operation to be recognized asa failed,
-	 * because the last data items are not processed properly.
-	 *
-	 * @throws Exception An exception in this method causes the operator to fail.
-	 */
-	@Override
-	public void close() throws Exception {}
-	
-	/**
-	 * This method is called at the very end of the operator's life, both in the case of a successful
-	 * completion of the operation, and in the case of a failure and canceling.
-	 *
-	 * This method is expected to make a thorough effort to release all resources
-	 * that the operator has acquired.
-	 */
-	@Override
-	public void dispose() {
-		if (keyValueStates != null) {
-			for (KvState<?, ?, ?> state : keyValueStates) {
-				state.dispose();
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Checkpointing
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
-		// here, we deal with key/value state snapshots
-		
-		StreamTaskState state = new StreamTaskState();
-		if (keyValueStates != null) {
-			HashMap<String, KvStateSnapshot<?, ?, ?>> snapshots = new HashMap<>(keyValueStatesByName.size());
-			
-			for (Map.Entry<String, KvState<?, ?, ?>> entry : keyValueStatesByName.entrySet()) {
-				KvStateSnapshot<?, ?, ?> snapshot = entry.getValue().shapshot(checkpointId, timestamp);
-				snapshots.put(entry.getKey(), snapshot);
-			}
-			
-			state.setKvStates(snapshots);
-		}
-		
-		return state;
-	}
-	
-	@Override
-	public void restoreState(StreamTaskState state) throws Exception {
-		// restore the key/value state. the actual restore happens lazily, when the function requests
-		// the state again, because the restore method needs information provided by the user function
-		keyValueStateSnapshots = state.getKvStates();
-	}
-	
-	@Override
-	public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception {
-		// by default, nothing needs a notification of checkpoint completion
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties and Services
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the execution config defined on the execution environment of the job to which this
-	 * operator belongs.
-	 * 
-	 * @return The job's execution config.
-	 */
-	public ExecutionConfig getExecutionConfig() {
-		return container.getExecutionConfig();
-	}
-	
-	public StreamConfig getOperatorConfig() {
-		return config;
-	}
-	
-	public StreamTask<?, ?> getContainingTask() {
-		return container;
-	}
-	
-	public ClassLoader getUserCodeClassloader() {
-		return container.getUserCodeClassLoader();
-	}
-	
-	/**
-	 * Returns a context that allows the operator to query information about the execution and also
-	 * to interact with systems such as broadcast variables and managed state. This also allows
-	 * to register timers.
-	 */
-	public StreamingRuntimeContext getRuntimeContext() {
-		return runtimeContext;
-	}
-
-	public StateBackend<?> getStateBackend() {
-		return container.getStateBackend();
-	}
-
-	/**
-	 * Register a timer callback. At the specified time the {@link Triggerable} will be invoked.
-	 * This call is guaranteed to not happen concurrently with method calls on the operator.
-	 *
-	 * @param time The absolute time in milliseconds.
-	 * @param target The target to be triggered.
-	 */
-	protected void registerTimer(long time, Triggerable target) {
-		container.registerTimer(time, target);
-	}
-
-	/**
-	 * Creates a key/value state handle, using the state backend configured for this task.
-	 *
-	 * @param stateType The type information for the state type, used for managed memory and state snapshots.
-	 * @param defaultValue The default value that the state should return for keys that currently have
-	 *                     no value associated with them 
-	 *
-	 * @param <V> The type of the state value.
-	 *
-	 * @return The key/value state for this operator.
-	 *
-	 * @throws IllegalStateException Thrown, if the key/value state was already initialized.
-	 * @throws Exception Thrown, if the state backend cannot create the key/value state.
-	 */
-	protected <V> OperatorState<V> createKeyValueState(
-			String name, TypeInformation<V> stateType, V defaultValue) throws Exception
-	{
-		return createKeyValueState(name, stateType.createSerializer(getExecutionConfig()), defaultValue);
-	}
-	
-	/**
-	 * Creates a key/value state handle, using the state backend configured for this task.
-	 * 
-	 * @param valueSerializer The type serializer for the state type, used for managed memory and state snapshots.
-	 * @param defaultValue The default value that the state should return for keys that currently have
-	 *                     no value associated with them 
-	 * 
-	 * @param <K> The type of the state key.
-	 * @param <V> The type of the state value.
-	 * @param <Backend> The type of the state backend that creates the key/value state.
-	 * 
-	 * @return The key/value state for this operator.
-	 * 
-	 * @throws IllegalStateException Thrown, if the key/value state was already initialized.
-	 * @throws Exception Thrown, if the state backend cannot create the key/value state.
-	 */
-	@SuppressWarnings({"rawtypes", "unchecked"})
-	protected <K, V, Backend extends StateBackend<Backend>> OperatorState<V> createKeyValueState(
-			String name, TypeSerializer<V> valueSerializer, V defaultValue) throws Exception
-	{
-		if (name == null || name.isEmpty()) {
-			throw new IllegalArgumentException();
-		}
-		if (keyValueStatesByName != null && keyValueStatesByName.containsKey(name)) {
-			throw new IllegalStateException("The key/value state has already been created");
-		}
-
-		TypeSerializer<K> keySerializer;
-		
-		// first time state access, make sure we load the state partitioner
-		if (stateKeySelector == null) {
-			stateKeySelector = config.getStatePartitioner(getUserCodeClassloader());
-			if (stateKeySelector == null) {
-				throw new UnsupportedOperationException("The function or operator is not executed " +
-						"on a KeyedStream and can hence not access the key/value state");
-			}
-
-			keySerializer = config.getStateKeySerializer(getUserCodeClassloader());
-			if (keySerializer == null) {
-				throw new Exception("State key serializer has not been configured in the config.");
-			}
-			this.keySerializer = keySerializer;
-		}
-		else if (this.keySerializer != null) {
-			keySerializer = (TypeSerializer<K>) this.keySerializer;
-		}
-		else {
-			// should never happen, this is merely a safeguard
-			throw new RuntimeException();
-		}
-		
-		@SuppressWarnings("unchecked")
-		Backend stateBackend = (Backend) container.getStateBackend();
-
-		KvState<K, V, Backend> kvstate = null;
-		
-		// check whether we restore the key/value state from a snapshot, or create a new blank one
-		if (keyValueStateSnapshots != null) {
-			@SuppressWarnings("unchecked")
-			KvStateSnapshot<K, V, Backend> snapshot = (KvStateSnapshot<K, V, Backend>) keyValueStateSnapshots.remove(name);
-
-			if (snapshot != null) {
-				kvstate = snapshot.restoreState(
-						stateBackend, keySerializer, valueSerializer, defaultValue, getUserCodeClassloader());
-			}
-		}
-		
-		if (kvstate == null) {
-			// create a new blank key/value state
-			kvstate = stateBackend.createKvState(keySerializer, valueSerializer, defaultValue);
-		}
-
-		if (keyValueStatesByName == null) {
-			keyValueStatesByName = new HashMap<>();
-		}
-		keyValueStatesByName.put(name, kvstate);
-		keyValueStates = keyValueStatesByName.values().toArray(new KvState[keyValueStatesByName.size()]);
-		return kvstate;
-	}
-	
-	@Override
-	@SuppressWarnings({"unchecked", "rawtypes"})
-	public void setKeyContextElement(StreamRecord record) throws Exception {
-		if (stateKeySelector != null && keyValueStates != null) {
-			KeySelector selector = stateKeySelector;
-			for (KvState kv : keyValueStates) {
-				kv.setCurrentKey(selector.getKey(record.getValue()));
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Context and chaining properties
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		this.chainingStrategy = strategy;
-	}
-	
-	@Override
-	public final ChainingStrategy getChainingStrategy() {
-		return chainingStrategy;
-	}
-	
-	@Override
-	public boolean isInputCopyingDisabled() {
-		return inputCopyDisabled;
-	}
-
-	/**
-	 * Enable object-reuse for this operator instance. This overrides the setting in
-	 * the {@link org.apache.flink.api.common.ExecutionConfig}
-	 */
-	public void disableInputCopy() {
-		this.inputCopyDisabled = true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
deleted file mode 100644
index 17bd08d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * This is used as the base class for operators that have a user-defined
- * function. This class handles the opening and closing of the user-defined functions,
- * as part of the operator life cycle.
- * 
- * @param <OUT>
- *            The output type of the operator
- * @param <F>
- *            The type of the user function
- */
-public abstract class AbstractUdfStreamOperator<OUT, F extends Function> extends AbstractStreamOperator<OUT> {
-
-	private static final long serialVersionUID = 1L;
-	
-	
-	/** the user function */
-	protected final F userFunction;
-	
-	/** Flag to prevent duplicate function.close() calls in close() and dispose() */
-	private transient boolean functionsClosed = false;
-	
-	
-	public AbstractUdfStreamOperator(F userFunction) {
-		this.userFunction = requireNonNull(userFunction);
-	}
-
-	/**
-	 * Gets the user function executed in this operator.
-	 * @return The user function of this operator.
-	 */
-	public F getUserFunction() {
-		return userFunction;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  operator life cycle
-	// ------------------------------------------------------------------------
-
-
-	@Override
-	public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output) {
-		super.setup(containingTask, config, output);
-		
-		FunctionUtils.setFunctionRuntimeContext(userFunction, getRuntimeContext());
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-		
-		FunctionUtils.openFunction(userFunction, new Configuration());
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		functionsClosed = true;
-		FunctionUtils.closeFunction(userFunction);
-	}
-
-	@Override
-	public void dispose() {
-		if (!functionsClosed) {
-			functionsClosed = true;
-			try {
-				FunctionUtils.closeFunction(userFunction);
-			}
-			catch (Throwable t) {
-				LOG.error("Exception while closing user function while failing or canceling task", t);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  checkpointing and recovery
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
-		StreamTaskState state = super.snapshotOperatorState(checkpointId, timestamp);
-
-		if (userFunction instanceof Checkpointed) {
-			@SuppressWarnings("unchecked")
-			Checkpointed<Serializable> chkFunction = (Checkpointed<Serializable>) userFunction;
-			
-			Serializable udfState;
-			try {
-				udfState = chkFunction.snapshotState(checkpointId, timestamp);
-			} 
-			catch (Exception e) {
-				throw new Exception("Failed to draw state snapshot from function: " + e.getMessage(), e);
-			}
-			
-			if (udfState != null) {
-				try {
-					StateBackend<?> stateBackend = getStateBackend();
-					StateHandle<Serializable> handle = 
-							stateBackend.checkpointStateSerializable(udfState, checkpointId, timestamp);
-					state.setFunctionState(handle);
-				}
-				catch (Exception e) {
-					throw new Exception("Failed to add the state snapshot of the function to the checkpoint: "
-							+ e.getMessage(), e);
-				}
-			}
-		}
-		
-		return state;
-	}
-
-	@Override
-	public void restoreState(StreamTaskState state) throws Exception {
-		super.restoreState(state);
-		
-		StateHandle<Serializable> stateHandle =  state.getFunctionState();
-		
-		if (userFunction instanceof Checkpointed && stateHandle != null) {
-			@SuppressWarnings("unchecked")
-			Checkpointed<Serializable> chkFunction = (Checkpointed<Serializable>) userFunction;
-			
-			Serializable functionState = stateHandle.getState(getUserCodeClassloader());
-			if (functionState != null) {
-				try {
-					chkFunction.restoreState(functionState);
-				}
-				catch (Exception e) {
-					throw new Exception("Failed to restore state to function: " + e.getMessage(), e);
-				}
-			}
-		}
-	}
-
-	@Override
-	public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception {
-		super.notifyOfCompletedCheckpoint(checkpointId);
-
-		if (userFunction instanceof CheckpointNotifier) {
-			((CheckpointNotifier) userFunction).notifyCheckpointComplete(checkpointId);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * 
-	 * Since the streaming API does not implement any parametrization of functions via a
-	 * configuration, the config returned here is actually empty.
-	 * 
-	 * @return The user function parameters (currently empty)
-	 */
-	public Configuration getUserFunctionParameters() {
-		return new Configuration();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java
deleted file mode 100644
index 3a752b0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-/**
- * Defines the chaining scheme for the operator.
- * By default {@link #ALWAYS} is used, which means operators will be eagerly chained whenever possible.
- */
-public enum ChainingStrategy {
-
-	/**
-	 * Chaining will happen even if chaining is disabled on the execution environment.
-	 * This should only be used by system-level operators, not operators implemented by users.
-	 */
-	FORCE_ALWAYS,
-
-	/** 
-	 * Operators will be eagerly chained whenever possible, for
-	 * maximal performance. It is generally a good practice to allow maximal
-	 * chaining and increase operator parallelism
-	 */
-	ALWAYS,
-
-	/**
-	 * The operator will not be chained to the preceding or succeeding operators.
-	 */
-	NEVER,
-	
-	
-	HEAD
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java
deleted file mode 100644
index 705c1b3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Interface for stream operators with one input. Use
- * {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator} as a base class if
- * you want to implement a custom operator.
- * 
- * @param <IN> The input type of the operator
- * @param <OUT> The output type of the operator
- */
-public interface OneInputStreamOperator<IN, OUT> extends StreamOperator<OUT> {
-
-	/**
-	 * Processes one element that arrived at this operator.
-	 * This method is guaranteed to not be called concurrently with other methods of the operator.
-	 */
-	void processElement(StreamRecord<IN> element) throws Exception;
-
-	/**
-	 * Processes a {@link Watermark}.
-	 * This method is guaranteed to not be called concurrently with other methods of the operator.
-	 *
-	 * @see org.apache.flink.streaming.api.watermark.Watermark
-	 */
-	void processWatermark(Watermark mark) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java
deleted file mode 100644
index 0cbc954..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.util.Collector;
-
-/**
- * A {@link org.apache.flink.streaming.api.operators.StreamOperator} is supplied with an object
- * of this interface that can be used to emit elements and other messages, such as barriers
- * and watermarks, from an operator.
- *
- * @param <T> The type of the elements that can be emitted.
- */
-public interface Output<T> extends Collector<T> {
-
-	/**
-	 * Emits a {@link Watermark} from an operator. This watermark is broadcast to all downstream
-	 * operators.
-	 *
-	 * <p>A watermark specifies that no element with a timestamp older or equal to the watermark
-	 * timestamp will be emitted in the future.
-	 */
-	void emitWatermark(Watermark mark);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OutputTypeConfigurable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OutputTypeConfigurable.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OutputTypeConfigurable.java
deleted file mode 100644
index 1d05966..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OutputTypeConfigurable.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-/**
- * Stream operators can implement this interface if they need access to the output type information
- * at {@link org.apache.flink.streaming.api.graph.StreamGraph} generation. This can be useful for
- * cases where the output type is specified by the returns method and, thus, after the stream
- * operator has been created.
- */
-public interface OutputTypeConfigurable<OUT> {
-
-	/**
-	 * Is called by the {@link org.apache.flink.streaming.api.graph.StreamGraph#addOperator(Integer, StreamOperator, TypeInformation, TypeInformation, String)}
-	 * method when the {@link org.apache.flink.streaming.api.graph.StreamGraph} is generated. The
-	 * method is called with the output {@link TypeInformation} which is also used for the
-	 * {@link org.apache.flink.streaming.runtime.tasks.StreamTask} output serializer.
-	 *
-	 * @param outTypeInfo Output type information of the {@link org.apache.flink.streaming.runtime.tasks.StreamTask}
-	 * @param executionConfig Execution configuration
-	 */
-	void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig);
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java
deleted file mode 100644
index 23b638e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class StreamFlatMap<IN, OUT>
-		extends AbstractUdfStreamOperator<OUT, FlatMapFunction<IN, OUT>>
-		implements OneInputStreamOperator<IN, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	private transient TimestampedCollector<OUT> collector;
-
-	public StreamFlatMap(FlatMapFunction<IN, OUT> flatMapper) {
-		super(flatMapper);
-		chainingStrategy = ChainingStrategy.ALWAYS;
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-		collector = new TimestampedCollector<OUT>(output);
-	}
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		collector.setTimestamp(element.getTimestamp());
-		userFunction.flatMap(element.getValue(), collector);
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
deleted file mode 100644
index cf6b489..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class StreamGroupedFold<IN, OUT, KEY>
-		extends AbstractUdfStreamOperator<OUT, FoldFunction<IN, OUT>>
-		implements OneInputStreamOperator<IN, OUT>, OutputTypeConfigurable<OUT> {
-
-	private static final long serialVersionUID = 1L;
-	
-	private static final String STATE_NAME = "_op_state";
-
-	// Grouped values
-	private transient OperatorState<OUT> values;
-	
-	private transient OUT initialValue;
-	
-	// Initial value serialization
-	private byte[] serializedInitialValue;
-	
-	private TypeSerializer<OUT> outTypeSerializer;
-	
-	public StreamGroupedFold(FoldFunction<IN, OUT> folder, OUT initialValue) {
-		super(folder);
-		this.initialValue = initialValue;
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-
-		if (serializedInitialValue == null) {
-			throw new RuntimeException("No initial value was serialized for the fold " +
-					"operator. Probably the setOutputType method was not called.");
-		}
-
-		ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
-		InputViewDataInputStreamWrapper in = new InputViewDataInputStreamWrapper(
-				new DataInputStream(bais)
-		);
-		initialValue = outTypeSerializer.deserialize(in);
-		values = createKeyValueState(STATE_NAME, outTypeSerializer, null);
-	}
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		OUT value = values.value();
-
-		if (value != null) {
-			OUT folded = userFunction.fold(outTypeSerializer.copy(value), element.getValue());
-			values.update(folded);
-			output.collect(element.replace(folded));
-		} else {
-			OUT first = userFunction.fold(outTypeSerializer.copy(initialValue), element.getValue());
-			values.update(first);
-			output.collect(element.replace(first));
-		}
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-
-	@Override
-	public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) {
-		outTypeSerializer = outTypeInfo.createSerializer(executionConfig);
-
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		OutputViewDataOutputStreamWrapper out = new OutputViewDataOutputStreamWrapper(
-				new DataOutputStream(baos)
-		);
-
-		try {
-			outTypeSerializer.serialize(initialValue, out);
-		} catch (IOException ioe) {
-			throw new RuntimeException("Unable to serialize initial value of type " +
-					initialValue.getClass().getSimpleName() + " of fold operator.", ioe);
-		}
-
-		serializedInitialValue = baos.toByteArray();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
deleted file mode 100644
index ae15e92..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class StreamGroupedReduce<IN> extends AbstractUdfStreamOperator<IN, ReduceFunction<IN>>
-		implements OneInputStreamOperator<IN, IN> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final String STATE_NAME = "_op_state";
-	
-	private transient OperatorState<IN> values;
-	
-	private TypeSerializer<IN> serializer;
-
-	
-	public StreamGroupedReduce(ReduceFunction<IN> reducer, TypeSerializer<IN> serializer) {
-		super(reducer);
-		this.serializer = serializer;
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-		values = createKeyValueState(STATE_NAME, serializer, null);
-	}
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		IN value = element.getValue();
-		IN currentValue = values.value();
-		
-		if (currentValue != null) {
-			IN reduced = userFunction.reduce(currentValue, value);
-			values.update(reduced);
-			output.collect(element.replace(reduced));
-		} else {
-			values.update(value);
-			output.collect(element.replace(value));
-		}
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-
-
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
deleted file mode 100644
index fac26f1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.io.Serializable;
-
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-
-/**
- * Basic interface for stream operators. Implementers would implement one of
- * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator} or
- * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator} to create operators
- * that process elements.
- * 
- * <p> The class {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator}
- * offers default implementation for the lifecycle and properties methods.
- *
- * <p> Methods of {@code StreamOperator} are guaranteed not to be called concurrently. Also, if using
- * the timer service, timer callbacks are also guaranteed not to be called concurrently with
- * methods on {@code StreamOperator}.
- * 
- * @param <OUT> The output type of the operator
- */
-public interface StreamOperator<OUT> extends Serializable {
-	
-	// ------------------------------------------------------------------------
-	//  life cycle
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Initializes the operator. Sets access to the context and the output.
-	 */
-	void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output);
-
-	/**
-	 * This method is called immediately before any elements are processed, it should contain the
-	 * operator's initialization logic.
-	 * 
-	 * @throws java.lang.Exception An exception in this method causes the operator to fail.
-	 */
-	void open() throws Exception;
-
-	/**
-	 * This method is called after all records have been added to the operators via the methods
-	 * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator#processElement(StreamRecord)}, or
-	 * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator#processElement1(StreamRecord)} and
-	 * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator#processElement2(StreamRecord)}.
-
-	 * <p>
-	 * The method is expected to flush all remaining buffered data. Exceptions during this flushing
-	 * of buffered should be propagated, in order to cause the operation to be recognized asa failed,
-	 * because the last data items are not processed properly.
-	 * 
-	 * @throws java.lang.Exception An exception in this method causes the operator to fail.
-	 */
-	void close() throws Exception;
-
-	/**
-	 * This method is called at the very end of the operator's life, both in the case of a successful
-	 * completion of the operation, and in the case of a failure and canceling.
-	 * 
-	 * This method is expected to make a thorough effort to release all resources
-	 * that the operator has acquired.
-	 */
-	void dispose();
-
-	// ------------------------------------------------------------------------
-	//  state snapshots
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Called to draw a state snapshot from the operator. This method snapshots the operator state
-	 * (if the operator is stateful) and the key/value state (if it is being used and has been
-	 * initialized).
-	 *
-	 * @param checkpointId The ID of the checkpoint.
-	 * @param timestamp The timestamp of the checkpoint.
-	 *
-	 * @return The StreamTaskState object, possibly containing the snapshots for the
-	 *         operator and key/value state.
-	 *
-	 * @throws Exception Forwards exceptions that occur while drawing snapshots from the operator
-	 *                   and the key/value state.
-	 */
-	StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception;
-	
-	/**
-	 * Restores the operator state, if this operator's execution is recovering from a checkpoint.
-	 * This method restores the operator state (if the operator is stateful) and the key/value state
-	 * (if it had been used and was initialized when the snapshot ocurred).
-	 *
-	 * <p>This method is called after {@link #setup(StreamTask, StreamConfig, Output)}
-	 * and before {@link #open()}.
-	 *
-	 * @param state The state of operator that was snapshotted as part of checkpoint
-	 *              from which the execution is restored.
-	 *
-	 * @throws Exception Exceptions during state restore should be forwarded, so that the system can
-	 *                   properly react to failed state restore and fail the execution attempt.
-	 */
-	void restoreState(StreamTaskState state) throws Exception;
-
-	/**
-	 * Called when the checkpoint with the given ID is completed and acknowledged on the JobManager.
-	 *
-	 * @param checkpointId The ID of the checkpoint that has been completed.
-	 *
-	 * @throws Exception Exceptions during checkpoint acknowledgement may be forwarded and will cause
-	 *                   the program to fail and enter recovery.
-	 */
-	void notifyOfCompletedCheckpoint(long checkpointId) throws Exception;
-
-	// ------------------------------------------------------------------------
-	//  miscellaneous
-	// ------------------------------------------------------------------------
-	
-	void setKeyContextElement(StreamRecord<?> record) throws Exception;
-	
-	/**
-	 * An operator can return true here to disable copying of its input elements. This overrides
-	 * the object-reuse setting on the {@link org.apache.flink.api.common.ExecutionConfig}
-	 */
-	boolean isInputCopyingDisabled();
-	
-	ChainingStrategy getChainingStrategy();
-
-	void setChainingStrategy(ChainingStrategy strategy);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java
deleted file mode 100644
index 1ce4ff6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.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.operators;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class StreamProject<IN, OUT extends Tuple>
-		extends AbstractStreamOperator<OUT>
-		implements OneInputStreamOperator<IN, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	private TypeSerializer<OUT> outSerializer;
-	private int[] fields;
-	private int numFields;
-
-	private transient OUT outTuple;
-
-	public StreamProject(int[] fields, TypeSerializer<OUT> outSerializer) {
-		this.fields = fields;
-		this.numFields = this.fields.length;
-		this.outSerializer = outSerializer;
-
-		chainingStrategy = ChainingStrategy.ALWAYS;
-	}
-
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		for (int i = 0; i < this.numFields; i++) {
-			outTuple.setField(((Tuple) element.getValue()).getField(fields[i]), i);
-		}
-		output.collect(element.replace(outTuple));
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-		outTuple = outSerializer.createInstance();
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java
deleted file mode 100644
index 6961a4d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSink.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class StreamSink<IN> extends AbstractUdfStreamOperator<Object, SinkFunction<IN>>
-		implements OneInputStreamOperator<IN, Object> {
-
-	private static final long serialVersionUID = 1L;
-
-	public StreamSink(SinkFunction<IN> sinkFunction) {
-		super(sinkFunction);
-
-		chainingStrategy = ChainingStrategy.ALWAYS;
-	}
-
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		userFunction.invoke(element.getValue());
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		// ignore it for now, we are a sink, after all
-	}
-}


[19/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java
deleted file mode 100644
index 9820ef8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/EvenOddOutputSelector.java
+++ /dev/null
@@ -1,31 +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.streaming.api.collector.selector.OutputSelector;
-
-import java.util.Arrays;
-
-public class EvenOddOutputSelector implements OutputSelector<Integer> {
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public Iterable<String> select(Integer value) {
-		return value % 2 == 0 ? Arrays.asList("even") : Arrays.asList("odd");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/FieldAccessorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/FieldAccessorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/FieldAccessorTest.java
deleted file mode 100644
index d35089a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/FieldAccessorTest.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-// This only tests a fraction of FieldAccessor. The other parts are tested indirectly by AggregationFunctionTest.
-public class FieldAccessorTest {
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void arrayFieldAccessorTest() {
-		int[] a = new int[]{3,5};
-		FieldAccessor<int[], Integer> fieldAccessor =
-				(FieldAccessor<int[], Integer>) (Object)
-						FieldAccessor.create(1, PrimitiveArrayTypeInfo.getInfoFor(a.getClass()), null);
-
-		assertEquals(Integer.class, fieldAccessor.getFieldType().getTypeClass());
-
-		assertEquals((Integer)a[1], fieldAccessor.get(a));
-
-		a = fieldAccessor.set(a, 6);
-		assertEquals((Integer)a[1], fieldAccessor.get(a));
-
-
-
-		Integer[] b = new Integer[]{3,5};
-		FieldAccessor<Integer[], Integer> fieldAccessor2 =
-				(FieldAccessor<Integer[], Integer>) (Object)
-						FieldAccessor.create(1, BasicArrayTypeInfo.getInfoFor(b.getClass()), null);
-
-		assertEquals(Integer.class, fieldAccessor2.getFieldType().getTypeClass());
-
-		assertEquals((Integer)b[1], fieldAccessor2.get(b));
-
-		b = fieldAccessor2.set(b, 6);
-		assertEquals((Integer)b[1], fieldAccessor2.get(b));
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void tupleFieldAccessorOutOfBoundsTest() {
-		try {
-			FieldAccessor<Tuple2<Integer, Integer>, Integer> fieldAccessor =
-					(FieldAccessor<Tuple2<Integer, Integer>, Integer>) (Object)
-							FieldAccessor.create(2, TupleTypeInfo.getBasicTupleTypeInfo(Integer.class, Integer.class),
-									null);
-			fail();
-		} catch (IndexOutOfBoundsException e) {
-			// Nothing to do here
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
deleted file mode 100644
index 0c708c6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
+++ /dev/null
@@ -1,159 +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 java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.operators.testutils.MockEnvironment;
-import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.mockito.stubbing.OngoingStubbing;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class MockContext<IN, OUT> {
-	
-	private List<OUT> outputs;
-
-	private MockOutput<OUT> output;
-
-	public MockContext(Collection<IN> inputs) {
-		if (inputs.isEmpty()) {
-			throw new RuntimeException("Inputs must not be empty");
-		}
-
-		outputs = new ArrayList<OUT>();
-		output = new MockOutput<OUT>(outputs);
-	}
-
-	public List<OUT> getOutputs() {
-		return outputs;
-	}
-
-	public Output<StreamRecord<OUT>> getOutput() {
-		return output;
-	}
-
-	public static <IN, OUT> List<OUT> createAndExecute(OneInputStreamOperator<IN, OUT> operator, List<IN> inputs) {
-		return createAndExecuteForKeyedStream(operator, inputs, null, null);
-	}
-	
-	public static <IN, OUT, KEY> List<OUT> createAndExecuteForKeyedStream(
-				OneInputStreamOperator<IN, OUT> operator, List<IN> inputs,
-				KeySelector<IN, KEY> keySelector, TypeInformation<KEY> keyType) {
-		
-		MockContext<IN, OUT> mockContext = new MockContext<IN, OUT>(inputs);
-
-		StreamConfig config = new StreamConfig(new Configuration());
-		if (keySelector != null && keyType != null) {
-			config.setStateKeySerializer(keyType.createSerializer(new ExecutionConfig()));
-			config.setStatePartitioner(keySelector);
-		}
-		
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		final Object lock = new Object();
-		final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-				
-		operator.setup(mockTask, config, mockContext.output);
-		try {
-			operator.open();
-
-			StreamRecord<IN> record = new StreamRecord<IN>(null);
-			for (IN in: inputs) {
-				record = record.replace(in);
-				synchronized (lock) {
-					operator.setKeyContextElement(record);
-					operator.processElement(record);
-				}
-			}
-
-			operator.close();
-		} catch (Exception e) {
-			throw new RuntimeException("Cannot invoke operator.", e);
-		} finally {
-			timerService.shutdownNow();
-		}
-
-		return mockContext.getOutputs();
-	}
-
-	private static StreamTask<?, ?> createMockTaskWithTimer(
-			final ScheduledExecutorService timerService, final Object lock)
-	{
-		StreamTask<?, ?> task = mock(StreamTask.class);
-		when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
-		when(task.getName()).thenReturn("Test task name");
-		when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
-		when(task.getEnvironment()).thenReturn(new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024));
-		when(task.getCheckpointLock()).thenReturn(lock);
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
-				final Long timestamp = (Long) invocationOnMock.getArguments()[0];
-				final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
-				timerService.schedule(
-						new Callable<Object>() {
-							@Override
-							public Object call() throws Exception {
-								synchronized (lock) {
-									target.trigger(timestamp);
-								}
-								return null;
-							}
-						},
-						timestamp - System.currentTimeMillis(),
-						TimeUnit.MILLISECONDS);
-				return null;
-			}
-		}).when(task).registerTimer(anyLong(), any(Triggerable.class));
-
-		// ugly Java generic hacks to get the generic state backend into the mock
-		@SuppressWarnings("unchecked")
-		OngoingStubbing<StateBackend<?>> stubbing =
-				(OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(task.getStateBackend());
-		stubbing.thenReturn(MemoryStateBackend.defaultInstance());
-		
-		return task;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java
deleted file mode 100644
index 5371ba0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockOutput.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class MockOutput<T> implements Output<StreamRecord<T>> {
-	private Collection<T> outputs;
-
-	public MockOutput(Collection<T> outputs) {
-		this.outputs = outputs;
-	}
-
-	@Override
-	public void collect(StreamRecord<T> record) {
-		T copied = SerializationUtils.deserialize(SerializationUtils
-				.serialize((Serializable) record.getValue()));
-		outputs.add(copied);
-	}
-
-	@Override
-	public void emitWatermark(Watermark mark) {
-		throw new RuntimeException("THIS MUST BE IMPLEMENTED");
-	}
-
-	@Override
-	public void close() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java
deleted file mode 100644
index bcb5691..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpIntMap.java
+++ /dev/null
@@ -1,28 +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.functions.MapFunction;
-
-public class NoOpIntMap implements MapFunction<Integer, Integer> {
-	private static final long serialVersionUID = 1L;
-
-	public Integer map(Integer value) throws Exception {
-		return value;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java
deleted file mode 100644
index d398121..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/NoOpSink.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-
-public final class NoOpSink<T> extends RichSinkFunction<T> {
-	public void invoke(T tuple) {
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
deleted file mode 100644
index 01f95bc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ /dev/null
@@ -1,159 +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.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.operators.testutils.MockEnvironment;
-import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.mockito.stubbing.OngoingStubbing;
-
-import java.util.Collection;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * A test harness for testing a {@link OneInputStreamOperator}.
- *
- * <p>
- * This mock task provides the operator with a basic runtime context and allows pushing elements
- * and watermarks into the operator. {@link java.util.Deque}s containing the emitted elements
- * and watermarks can be retrieved. You are free to modify these.
- */
-public class OneInputStreamOperatorTestHarness<IN, OUT> {
-
-	final OneInputStreamOperator<IN, OUT> operator;
-
-	final ConcurrentLinkedQueue<Object> outputList;
-
-	final StreamConfig config;
-	
-	final ExecutionConfig executionConfig;
-	
-	final Object checkpointLock;
-	
-	
-	public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator) {
-		this.operator = operator;
-		this.outputList = new ConcurrentLinkedQueue<Object>();
-		this.config = new StreamConfig(new Configuration());
-		this.executionConfig = new ExecutionConfig();
-		this.checkpointLock = new Object();
-
-		Environment env = new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024);
-		StreamTask<?, ?> mockTask = mock(StreamTask.class);
-		when(mockTask.getName()).thenReturn("Mock Task");
-		when(mockTask.getCheckpointLock()).thenReturn(checkpointLock);
-		when(mockTask.getConfiguration()).thenReturn(config);
-		when(mockTask.getEnvironment()).thenReturn(env);
-		when(mockTask.getExecutionConfig()).thenReturn(executionConfig);
-		
-		// ugly Java generic hacks
-		@SuppressWarnings("unchecked")
-		OngoingStubbing<StateBackend<?>> stubbing = 
-				(OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(mockTask.getStateBackend());
-		stubbing.thenReturn(MemoryStateBackend.defaultInstance());
-
-		operator.setup(mockTask, config, new MockOutput());
-	}
-
-	public <K> void configureForKeyedStream(KeySelector<IN, K> keySelector, TypeInformation<K> keyType) {
-		ClosureCleaner.clean(keySelector, false);
-		config.setStatePartitioner(keySelector);
-		config.setStateKeySerializer(keyType.createSerializer(executionConfig));
-	}
-	
-	/**
-	 * Get all the output from the task. This contains StreamRecords and Events interleaved. Use
-	 * {@link org.apache.flink.streaming.util.TestHarnessUtil#getStreamRecordsFromOutput(java.util.List)}
-	 * to extract only the StreamRecords.
-	 */
-	public ConcurrentLinkedQueue<Object> getOutput() {
-		return outputList;
-	}
-
-	/**
-	 * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open()}
-	 */
-	public void open() throws Exception {
-		operator.open();
-	}
-
-	/**
-	 * Calls close on the operator.
-	 */
-	public void close() throws Exception {
-		operator.close();
-	}
-
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		operator.setKeyContextElement(element);
-		operator.processElement(element);
-	}
-
-	public void processElements(Collection<StreamRecord<IN>> elements) throws Exception {
-		for (StreamRecord<IN> element: elements) {
-			operator.setKeyContextElement(element);
-			operator.processElement(element);
-		}
-	}
-
-	public void processWatermark(Watermark mark) throws Exception {
-		operator.processWatermark(mark);
-	}
-
-	private class MockOutput implements Output<StreamRecord<OUT>> {
-
-		private TypeSerializer<OUT> outputSerializer;
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			outputList.add(mark);
-		}
-
-		@Override
-		public void collect(StreamRecord<OUT> element) {
-			if (outputSerializer == null) {
-				outputSerializer = TypeExtractor.getForObject(element.getValue()).createSerializer(executionConfig);
-			}
-			outputList.add(new StreamRecord<OUT>(outputSerializer.copy(element.getValue()),
-					element.getTimestamp()));
-		}
-
-		@Override
-		public void close() {
-			// ignore
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java
deleted file mode 100644
index a46ff55..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/ReceiveCheckNoOpSink.java
+++ /dev/null
@@ -1,42 +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.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertTrue;
-
-public final class ReceiveCheckNoOpSink<T> extends RichSinkFunction<T> {
-	private List<T> received;
-
-	public void invoke(T tuple) {
-		received.add(tuple);
-	}
-
-	public void open(Configuration conf) {
-		received = new ArrayList<T>();
-	}
-
-	public void close() {
-		assertTrue(received.size() > 0);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketOutputTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketOutputTestBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketOutputTestBase.java
deleted file mode 100644
index 4ded0fa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketOutputTestBase.java
+++ /dev/null
@@ -1,130 +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.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.util.NetUtils;
-
-import org.junit.Assert;
-
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * Test base for streaming programs relying on an open server socket to write to.
- */
-public abstract class SocketOutputTestBase extends StreamingProgramTestBase {
-
-	protected static final String HOST = "localhost";
-	protected static Integer port;
-	protected Set<String> dataReadFromSocket = new HashSet<String>();
-
-	@Override
-	protected void preSubmit() throws Exception {
-		port = NetUtils.getAvailablePort();
-		temporarySocket = createLocalSocket(port);
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		Set<String> expectedData = new HashSet<String>(Arrays.asList(WordCountData.STREAMING_COUNTS_AS_TUPLES.split("\n")));
-		Assert.assertEquals(expectedData, dataReadFromSocket);
-		temporarySocket.close();
-	}
-
-	protected ServerSocket temporarySocket;
-
-	public ServerSocket createLocalSocket(int port) throws Exception {
-		ServerSocket serverSocket = new ServerSocket(port);
-		ServerThread st = new ServerThread(serverSocket);
-		st.start();
-		return serverSocket;
-	}
-
-	protected class ServerThread extends Thread {
-
-		private ServerSocket serverSocket;
-		private Thread t;
-
-		public ServerThread(ServerSocket serverSocket) {
-			this.serverSocket = serverSocket;
-			t = new Thread(this);
-		}
-
-		public void waitForAccept() throws Exception {
-			Socket socket = serverSocket.accept();
-			BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()));
-			DeserializationSchema<String> schema = new DummyStringSchema();
-			String rawData = in.readLine();
-			while (rawData != null){
-				String string = schema.deserialize(rawData.getBytes());
-				dataReadFromSocket.add(string);
-				rawData = in.readLine();
-			}
-			socket.close();
-		}
-
-		public void run() {
-			try {
-				waitForAccept();
-			} catch (Exception e) {
-				Assert.fail();
-				throw new RuntimeException(e);
-			}
-		}
-
-		@Override
-		public void start() {
-			t.start();
-		}
-	}
-
-	public static class DummyStringSchema implements DeserializationSchema<String>, SerializationSchema<String, byte[]>{
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean isEndOfStream(String nextElement) {
-		return nextElement.equals("q");
-	}
-
-		@Override
-		public byte[] serialize(String element) {
-		return element.getBytes();
-	}
-
-		@Override
-		public String deserialize(byte[] message) {
-		return new String(message);
-	}
-
-		@Override
-		public TypeInformation<String> getProducedType() {
-		return TypeExtractor.getForClass(String.class);
-	}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketProgramITCaseBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketProgramITCaseBase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketProgramITCaseBase.java
deleted file mode 100644
index d1bd64a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SocketProgramITCaseBase.java
+++ /dev/null
@@ -1,90 +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.test.testdata.WordCountData;
-import org.apache.flink.util.NetUtils;
-
-import org.junit.Assert;
-
-import java.io.PrintWriter;
-import java.net.ServerSocket;
-import java.net.Socket;
-
-public abstract class SocketProgramITCaseBase extends StreamingProgramTestBase {
-
-	protected static final String HOST = "localhost";
-	protected static Integer port;
-	protected String resultPath;
-
-	private ServerSocket temporarySocket;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		port = NetUtils.getAvailablePort();
-		temporarySocket = createSocket(HOST, port, WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath);
-		temporarySocket.close();
-	}
-
-	public ServerSocket createSocket(String host, int port, String contents) throws Exception {
-		ServerSocket serverSocket = new ServerSocket(port);
-		ServerThread st = new ServerThread(serverSocket, contents);
-		st.start();
-		return serverSocket;
-	}
-
-	private static class ServerThread extends Thread {
-
-		private ServerSocket serverSocket;
-		private String contents;
-		private Thread t;
-
-		public ServerThread(ServerSocket serverSocket, String contents) {
-			this.serverSocket = serverSocket;
-			this.contents = contents;
-			t = new Thread(this);
-		}
-
-		public void waitForAccept() throws Exception {
-			Socket socket = serverSocket.accept();
-			PrintWriter writer = new PrintWriter(socket.getOutputStream(), true);
-			writer.println(contents);
-			writer.close();
-			socket.close();
-		}
-
-		public void run() {
-			try {
-				waitForAccept();
-			} catch (Exception e) {
-				Assert.fail();
-			}
-		}
-
-		@Override
-		public void start() {
-			t.start();
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java
deleted file mode 100644
index 2afdc40..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.functions.RichFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.operators.testutils.MockEnvironment;
-import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-
-import static org.mockito.Mockito.*;
-
-public class SourceFunctionUtil<T> {
-
-	public static <T> List<T> runSourceFunction(SourceFunction<T> sourceFunction) throws Exception {
-		List<T> outputs = new ArrayList<T>();
-		
-		if (sourceFunction instanceof RichFunction) {
-
-			AbstractStreamOperator<?> operator = mock(AbstractStreamOperator.class);
-			when(operator.getExecutionConfig()).thenReturn(new ExecutionConfig());
-			
-			RuntimeContext runtimeContext =  new StreamingRuntimeContext(
-					operator,
-					new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024),
-					new HashMap<String, Accumulator<?, ?>>());
-			
-			((RichFunction) sourceFunction).setRuntimeContext(runtimeContext);
-
-			((RichFunction) sourceFunction).open(new Configuration());
-		}
-		try {
-			final Output<StreamRecord<T>> collector = new MockOutput<T>(outputs);
-			final Object lockingObject = new Object();
-			SourceFunction.SourceContext<T> ctx;
-			if (sourceFunction instanceof EventTimeSourceFunction) {
-				ctx = new StreamSource.ManualWatermarkContext<T>(lockingObject, collector);
-			} else {
-				ctx = new StreamSource.NonWatermarkContext<T>(lockingObject, collector);
-			}
-			sourceFunction.run(ctx);
-		} catch (Exception e) {
-			throw new RuntimeException("Cannot invoke source.", e);
-		}
-		return outputs;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 4e02f2c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.runtime.StreamingMode;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
-import org.apache.flink.test.util.TestBaseUtils;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-/**
- * Base class for streaming unit tests that run multiple tests and want to reuse the same
- * Flink cluster. This saves a significant amount of time, since the startup and
- * shutdown of the Flink clusters (including actor systems, etc) usually dominates
- * the execution of the actual tests.
- *
- * To write a unit test against this test base, simply extend it and add
- * one or more regular test methods and retrieve the StreamExecutionEnvironment from
- * the context:
- *
- * <pre>
- *   {@literal @}Test
- *   public void someTest() {
- *       StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- *       // test code
- *       env.execute();
- *   }
- *
- *   {@literal @}Test
- *   public void anotherTest() {
- *       StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- *       // test code
- *       env.execute();
- *   }
- *
- * </pre>
- */
-public class StreamingMultipleProgramsTestBase extends TestBaseUtils {
-
-	// ------------------------------------------------------------------------
-	//  The mini cluster that is shared across tests
-	// ------------------------------------------------------------------------
-
-	protected static final int DEFAULT_PARALLELISM = 4;
-
-	protected static ForkableFlinkMiniCluster cluster;
-	
-
-	// ------------------------------------------------------------------------
-	//  Cluster setup & teardown
-	// ------------------------------------------------------------------------
-
-	@BeforeClass
-	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/06f6ac5d/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
deleted file mode 100644
index ce3aa86..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/StreamingProgramTestBase.java
+++ /dev/null
@@ -1,111 +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.configuration.Configuration;
-import org.apache.flink.runtime.StreamingMode;
-import org.apache.flink.test.util.AbstractTestBase;
-
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-
-public abstract class StreamingProgramTestBase extends AbstractTestBase {
-
-	protected static final int DEFAULT_PARALLELISM = 4;
-
-	private int parallelism;
-	
-	
-	public StreamingProgramTestBase() {
-		super(new Configuration(), StreamingMode.STREAMING);
-		setParallelism(DEFAULT_PARALLELISM);
-	}
-
-
-	public void setParallelism(int parallelism) {
-		this.parallelism = parallelism;
-		setTaskManagerNumSlots(parallelism);
-	}
-	
-	public int getParallelism() {
-		return 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();
-
-			TestStreamEnvironment.setAsContext(this.executor, getParallelism());
-
-			// 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());
-			}
-			finally {
-				TestStreamEnvironment.unsetAsContext();
-			}
-
-			// post-submit
-			try {
-				postSubmit();
-			}
-			catch (Exception e) {
-				System.err.println(e.getMessage());
-				e.printStackTrace();
-				fail("Post-submit work caused an error: " + e.getMessage());
-			}
-		}
-		finally {
-			stopCluster();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
deleted file mode 100644
index 0c5cd8f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Assert;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Queue;
-
-/**
- * Utils for working with the various test harnesses.
- */
-public class TestHarnessUtil {
-	/**
-	 * Extracts the StreamRecords from the given output list.
-	 */
-	@SuppressWarnings("unchecked")
-	public static <OUT> List<StreamRecord<OUT>> getStreamRecordsFromOutput(List<Object> output) {
-		List<StreamRecord<OUT>> resultElements = new LinkedList<StreamRecord<OUT>>();
-		for (Object e: output) {
-			if (e instanceof StreamRecord) {
-				resultElements.add((StreamRecord<OUT>) e);
-			}
-		}
-		return resultElements;
-	}
-
-	/**
-	 * Extracts the raw elements from the given output list.
-	 */
-	@SuppressWarnings("unchecked")
-	public static <OUT> List<OUT> getRawElementsFromOutput(Queue<Object> output) {
-		List<OUT> resultElements = new LinkedList<OUT>();
-		for (Object e: output) {
-			if (e instanceof StreamRecord) {
-				resultElements.add(((StreamRecord<OUT>) e).getValue());
-			}
-		}
-		return resultElements;
-	}
-
-	/**
-	 * Compare the two queues containing operator/task output by converting them to an array first.
-	 */
-	public static void assertOutputEquals(String message, Queue<Object> expected, Queue<Object> actual) {
-		Assert.assertArrayEquals(message,
-				expected.toArray(),
-				actual.toArray());
-
-	}
-
-	/**
-	 * Compare the two queues containing operator/task output by converting them to an array first.
-	 */
-	public static void assertOutputEqualsSorted(String message, Queue<Object> expected, Queue<Object> actual, Comparator<Object> comparator) {
-		Object[] sortedExpected = expected.toArray();
-		Object[] sortedActual = actual.toArray();
-
-		Arrays.sort(sortedExpected, comparator);
-		Arrays.sort(sortedActual, comparator);
-
-		Assert.assertArrayEquals(message, sortedExpected, sortedActual);
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java
deleted file mode 100644
index 423d08e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListResultSink.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.TreeSet;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-
-public class TestListResultSink<T> extends RichSinkFunction<T> {
-
-	private static final long serialVersionUID = 1L;
-	private int resultListId;
-
-	public TestListResultSink() {
-		this.resultListId = TestListWrapper.getInstance().createList();
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-	}
-
-	@Override
-	public void invoke(T value) throws Exception {
-		synchronized (resultList()) {
-			resultList().add(value);
-		}
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-	}
-
-	@SuppressWarnings("unchecked")
-	private List<T> resultList() {
-		synchronized (TestListWrapper.getInstance()) {
-			return (List<T>) TestListWrapper.getInstance().getList(resultListId);
-		}
-	}
-
-	public List<T> getResult() {
-		synchronized (resultList()) {
-			ArrayList<T> copiedList = new ArrayList<T>(resultList());
-			return copiedList;
-		}
-	}
-
-	public List<T> getSortedResult() {
-		synchronized (resultList()) {
-			TreeSet<T> treeSet = new TreeSet<T>(resultList());
-			ArrayList<T> sortedList = new ArrayList<T>(treeSet);
-			return sortedList;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.java
deleted file mode 100644
index 751f836..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestListWrapper.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.util;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class TestListWrapper {
-
-	private static TestListWrapper instance;
-
-	@SuppressWarnings("rawtypes")
-	private List<List<? extends Comparable>> lists;
-
-	@SuppressWarnings("rawtypes")
-	private TestListWrapper() {
-		lists = Collections.synchronizedList(new ArrayList<List<? extends Comparable>>());
-	}
-
-	public static TestListWrapper getInstance() {
-		if (instance == null) {
-			instance = new TestListWrapper();
-		}
-		return instance;
-	}
-
-	/**
-	 * Creates and stores a list, returns with the id.
-	 *
-	 * @return The ID of the list.
-	 */
-	@SuppressWarnings("rawtypes")
-	public int createList() {
-		lists.add(new ArrayList<Comparable>());
-		return lists.size() - 1;
-	}
-
-	public List<?> getList(int listId) {
-		@SuppressWarnings("rawtypes")
-		List<? extends Comparable> list = lists.get(listId);
-		if (list == null) {
-			throw new RuntimeException("No such list.");
-		}
-
-		return list;
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 8cd1e4a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import com.google.common.base.Preconditions;
-import org.apache.flink.api.common.JobExecutionResult;
-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 {
-	
-	/** The mini cluster in which this environment executes its jobs */
-	private ForkableFlinkMiniCluster executor;
-	
-
-	public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism) {
-		this.executor = Preconditions.checkNotNull(executor);
-		setParallelism(parallelism);
-	}
-	
-	@Override
-	public JobExecutionResult execute(String jobName) throws Exception {
-		final JobGraph jobGraph = getStreamGraph().getJobGraph(jobName);
-		return executor.submitJobAndWait(jobGraph, false);
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * 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 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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
deleted file mode 100644
index c586db3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.operators.testutils.MockEnvironment;
-import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.mockito.stubbing.OngoingStubbing;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * A test harness for testing a {@link TwoInputStreamOperator}.
- *
- * <p>
- * This mock task provides the operator with a basic runtime context and allows pushing elements
- * and watermarks into the operator. {@link java.util.Deque}s containing the emitted elements
- * and watermarks can be retrieved. you are free to modify these.
- */
-public class TwoInputStreamOperatorTestHarness<IN1, IN2, OUT> {
-
-	TwoInputStreamOperator<IN1, IN2, OUT> operator;
-
-	final ConcurrentLinkedQueue<Object> outputList;
-
-	final ExecutionConfig executionConfig;
-
-	final Object checkpointLock;
-
-	public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator<IN1, IN2, OUT> operator) {
-		this(operator, new StreamConfig(new Configuration()));
-	}
-		
-	public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator<IN1, IN2, OUT> operator, StreamConfig config) {
-		this.operator = operator;
-		this.outputList = new ConcurrentLinkedQueue<Object>();
-		this.executionConfig = new ExecutionConfig();
-		this.checkpointLock = new Object();
-
-		Environment env = new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024);
-		StreamTask<?, ?> mockTask = mock(StreamTask.class);
-		when(mockTask.getName()).thenReturn("Mock Task");
-		when(mockTask.getCheckpointLock()).thenReturn(checkpointLock);
-		when(mockTask.getConfiguration()).thenReturn(config);
-		when(mockTask.getEnvironment()).thenReturn(env);
-		when(mockTask.getExecutionConfig()).thenReturn(executionConfig);
-
-		// ugly Java generic hacks
-		@SuppressWarnings("unchecked")
-		OngoingStubbing<StateBackend<?>> stubbing =
-				(OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(mockTask.getStateBackend());
-		stubbing.thenReturn(MemoryStateBackend.defaultInstance());
-
-		operator.setup(mockTask, new StreamConfig(new Configuration()), new MockOutput());
-	}
-
-	/**
-	 * Get all the output from the task. This contains StreamRecords and Events interleaved. Use
-	 * {@link org.apache.flink.streaming.util.TestHarnessUtil#getStreamRecordsFromOutput(java.util.List)}
-	 * to extract only the StreamRecords.
-	 */
-	public ConcurrentLinkedQueue<Object> getOutput() {
-		return outputList;
-	}
-
-
-	/**
-	 * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open()}.
-	 */
-	public void open() throws Exception {
-		operator.open();
-	}
-
-	/**
-	 * Calls close on the operator.
-	 */
-	public void close() throws Exception {
-		operator.close();
-	}
-
-	public void processElement1(StreamRecord<IN1> element) throws Exception {
-		operator.processElement1(element);
-	}
-
-	public void processElement2(StreamRecord<IN2> element) throws Exception {
-		operator.processElement2(element);
-	}
-
-	public void processWatermark1(Watermark mark) throws Exception {
-		operator.processWatermark1(mark);
-	}
-
-	public void processWatermark2(Watermark mark) throws Exception {
-		operator.processWatermark2(mark);
-	}
-
-	private class MockOutput implements Output<StreamRecord<OUT>> {
-
-		private TypeSerializer<OUT> outputSerializer;
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public void emitWatermark(Watermark mark) {
-			outputList.add(mark);
-		}
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public void collect(StreamRecord<OUT> element) {
-			if (outputSerializer == null) {
-				outputSerializer = TypeExtractor.getForObject(element.getValue()).createSerializer(executionConfig);
-			}
-			outputList.add(new StreamRecord<>(outputSerializer.copy(element.getValue()),
-					element.getTimestamp()));
-		}
-
-		@Override
-		public void close() {
-			// ignore
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java
deleted file mode 100644
index 1c0f850..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TypeInformationSerializationSchemaTest.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.core.testutils.CommonTestUtils;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-public class TypeInformationSerializationSchemaTest {
-	
-	@Test
-	public void testDeSerialization() {
-		try {
-			TypeInformation<MyPOJO> info = TypeExtractor.getForClass(MyPOJO.class);
-			
-			TypeInformationSerializationSchema<MyPOJO> schema =
-					new TypeInformationSerializationSchema<MyPOJO>(info, new ExecutionConfig());
-			
-			MyPOJO[] types = {
-					new MyPOJO(72, new Date(763784523L), new Date(88234L)),
-					new MyPOJO(-1, new Date(11111111111111L)),
-					new MyPOJO(42),
-					new MyPOJO(17, new Date(222763784523L))
-			};
-			
-			for (MyPOJO val : types) {
-				byte[] serialized = schema.serialize(val);
-				MyPOJO deser = schema.deserialize(serialized);
-				assertEquals(val, deser);
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSerializability() {
-		try {
-			TypeInformation<MyPOJO> info = TypeExtractor.getForClass(MyPOJO.class);
-			TypeInformationSerializationSchema<MyPOJO> schema =
-					new TypeInformationSerializationSchema<MyPOJO>(info, new ExecutionConfig());
-
-			// this needs to succeed
-			CommonTestUtils.createCopySerializable(schema);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Test data types
-	// ------------------------------------------------------------------------
-	
-	public static class MyPOJO {
-		
-		public int aField;
-		public List<Date> aList;
-		
-		public MyPOJO() {}
-
-		public MyPOJO(int iVal, Date... dates) {
-			this.aField = iVal;
-			this.aList = new ArrayList<>(Arrays.asList(dates));
-		}
-
-		@Override
-		public int hashCode() {
-			return aField;
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj instanceof MyPOJO) {
-				MyPOJO that = (MyPOJO) obj;
-				return this.aField == that.aField && (this.aList == null ? 
-						that.aList == null :
-						that.aList != null && this.aList.equals(that.aList));
-			}
-			return super.equals(obj);
-		}
-
-		@Override
-		public String toString() {
-			return String.format("MyPOJO " + aField + " " + aList);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java
deleted file mode 100644
index 63375a7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java
+++ /dev/null
@@ -1,80 +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.keys;
-
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class ArrayKeySelectorTest {
-
-	@Test
-	public void testObjectArrays() {
-		try {
-			String[] array1 = { "a", "b", "c", "d", "e" };
-			String[] array2 = { "v", "w", "x", "y", "z" };
-			
-			KeySelectorUtil.ArrayKeySelector<String[]> singleFieldSelector =
-					KeySelectorUtil.getSelectorForArray(new int[] {1}, BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO);
-			
-			assertEquals(new Tuple1<>("b"), singleFieldSelector.getKey(array1));
-			assertEquals(new Tuple1<>("w"), singleFieldSelector.getKey(array2));
-
-			KeySelectorUtil.ArrayKeySelector<String[]> twoFieldsSelector =
-					KeySelectorUtil.getSelectorForArray(new int[] {3, 0}, BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO);
-			
-			assertEquals(new Tuple2<>("d", "a"), twoFieldsSelector.getKey(array1));
-			assertEquals(new Tuple2<>("y", "v"), twoFieldsSelector.getKey(array2));
-			
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testPrimitiveArrays() {
-		try {
-			int[] array1 = { 1, 2, 3, 4, 5 };
-			int[] array2 = { -5, -4, -3, -2, -1, 0 };
-
-			KeySelectorUtil.ArrayKeySelector<int[]> singleFieldSelector = 
-					KeySelectorUtil.getSelectorForArray(new int[] {1}, PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO);
-
-			assertEquals(new Tuple1<>(2), singleFieldSelector.getKey(array1));
-			assertEquals(new Tuple1<>(-4), singleFieldSelector.getKey(array2));
-
-			KeySelectorUtil.ArrayKeySelector<int[]> twoFieldsSelector =
-					KeySelectorUtil.getSelectorForArray(new int[] {3, 0}, PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO);
-			
-			assertEquals(new Tuple2<>(4, 1), twoFieldsSelector.getKey(array1));
-			assertEquals(new Tuple2<>(-2, -5), twoFieldsSelector.getKey(array2));
-
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j-test.properties
deleted file mode 100644
index 0b686e5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-# Set root logger level to DEBUG and its only appender to A1.
-log4j.rootLogger=OFF, A1
-
-# A1 is set to be a ConsoleAppender.
-log4j.appender.A1=org.apache.log4j.ConsoleAppender
-
-# A1 uses PatternLayout.
-log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j.properties b/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j.properties
deleted file mode 100644
index ed2bbcb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-# This file ensures that tests executed from the IDE show log output
-
-log4j.rootLogger=OFF, console
-
-# Log all infos in the given file
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target = System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml b/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml
deleted file mode 100644
index 4f56748..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +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.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
-</configuration>
\ No newline at end of file


[44/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/pom.xml b/flink-staging/flink-streaming/flink-streaming-core/pom.xml
deleted file mode 100644
index cebb538..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/pom.xml
+++ /dev/null
@@ -1,116 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-streaming-core</artifactId>
-	<name>flink-streaming-core</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-runtime</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.commons</groupId>
-			<artifactId>commons-math</artifactId>
-			<version>2.2</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.sling</groupId>
-			<artifactId>org.apache.sling.commons.json</artifactId>
-			<version>2.0.6</version>
-		</dependency>
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-math3</artifactId>
-            <version>3.5</version>
-        </dependency>
-
-    </dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- disable fork reuse for the streaming project, because of
-			incorrect declaration of tests -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<reuseForks>false</reuseForks>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java
deleted file mode 100644
index db46d00..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api;
-
-/**
- * The checkpointing mode defines what consistency guarantees the system gives in the presence of
- * failures.
- * 
- * <p>When checkpointing is activated, the data streams are replayed such that lost parts of the
- * processing are repeated. For stateful operations and functions, the checkpointing mode defines
- * whether the system draws checkpoints such that a recovery behaves as if the operators/functions
- * see each record "exactly once" ({@link #EXACTLY_ONCE}), or whether the checkpoints are drawn
- * in a simpler fashion that typically encounteres some duplicates upon recovery
- * ({@link #AT_LEAST_ONCE})</p> 
- */
-public enum CheckpointingMode {
-
-	/**
-	 * Sets the checkpointing mode to "exactly once". This mode means that the system will
-	 * checkpoint the operator and user function state in such a way that, upon recovery,
-	 * every record will be reflected exactly once in the operator state.
-	 * 
-	 * <p>For example, if a user function counts the number of elements in a stream, 
-	 * this number will consistently be equal to the number of actual elements in the stream,
-	 * regardless of failures and recovery.</p>
-	 * 
-	 * <p>Note that this does not mean that each record flows through the streaming data flow
-	 * only once. It means that upon recovery, the state of operators/functions is restored such
-	 * that the resumed data streams pick up exactly at after the last modification to the state.</p> 
-	 *  
-	 * <p>Note that this mode does not guarantee exactly-once behavior in the interaction with
-	 * external systems (only state in Flink's operators and user functions). The reason for that
-	 * is that a certain level of "collaboration" is required between two systems to achieve
-	 * exactly-once guarantees. However, for certain systems, connectors can be written that facilitate
-	 * this collaboration.</p>
-	 * 
-	 * <p>This mode sustains high throughput. Depending on the data flow graph and operations,
-	 * this mode may increase the record latency, because operators need to align their input
-	 * streams, in order to create a consistent snapshot point. The latency increase for simple
-	 * dataflows (no repartitioning) is negligible. For simple dataflows with repartitioning, the average
-	 * latency remains small, but the slowest records typically have an increased latency.</p>
-	 */
-	EXACTLY_ONCE,
-
-	/**
-	 * Sets the checkpointing mode to "at least once". This mode means that the system will
-	 * checkpoint the operator and user function state in a simpler way. Upon failure and recovery,
-	 * some records may be reflected multiple times in the operator state.
-	 * 
-	 * <p>For example, if a user function counts the number of elements in a stream, 
-	 * this number will equal to, or larger, than the actual number of elements in the stream,
-	 * in the presence of failure and recovery.</p>
-	 * 
-	 * <p>This mode has minimal impact on latency and may be preferable in very-low latency
-	 * scenarios, where a sustained very-low latency (such as few milliseconds) is needed,
-	 * and where occasional duplicate messages (on recovery) do not matter.</p>
-	 */
-	AT_LEAST_ONCE
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java
deleted file mode 100644
index 125ca65..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api;
-
-/**
- * The time characteristic defines how the system determines time for time-dependent
- * order and operations that depend on time (such as time windows).
- */
-public enum TimeCharacteristic {
-
-	/**
-	 * Processing time for operators means that the operator uses the system clock of the machine
-	 * to determine the current time of the data stream. Processing-time windows trigger based
-	 * on wall-clock time and include whatever elements happen to have arrived at the operator at
-	 * that point in time.
-	 * <p>
-	 * Using processing time for window operations results in general in quite non-deterministic results,
-	 * because the contents of the windows depends on the speed in which elements arrive. It is, however,
-	 * the cheapest method of forming windows and the method that introduces the least latency.
-	 */
-	ProcessingTime,
-
-	/**
-	 * Ingestion time means that the time of each individual element in the stream is determined
-	 * when the element enters the Flink streaming data flow. Operations like windows group the
-	 * elements based on that time, meaning that processing speed within the streaming dataflow
-	 * does not affect windowing, but only the speed at which sources receive elements.
-	 * <p>
-	 * Ingestion time is often a good compromise between processing time and event time.
-	 * It does not need and special manual form of watermark generation, and events are typically
-	 * not too much out-or-order when they arrive at operators; in fact, out-of-orderness can 
-	 * only be introduced by streaming shuffles or split/join/union operations. The fact that elements
-	 * are not very much out-of-order means that the latency increase is moderate, compared to event
-	 * time.
-	 */
-	IngestionTime,
-
-	/**
-	 * Event time means that the time of each individual element in the stream (also called event)
-	 * is determined by the event's individual custom timestamp. These timestamps either exist in the
-	 * elements from before they entered the Flink streaming dataflow, or are user-assigned at the sources.
-	 * The big implication of this is that elements arrive in the sources and in all operators generally
-	 * out of order, meaning that elements with earlier timestamps may arrive after elements with
-	 * later timestamps.
-	 * <p>
-	 * Operators that window or order data with respect to event time must buffer data until they can
-	 * be sure that all timestamps for a certain time interval have been received. This is handled by
-	 * the so called "time watermarks".
-	 * <p>
-	 * Operations based on event time are very predictable - the result of windowing operations
-	 * is typically identical no matter when the window is executed and how fast the streams operate.
-	 * At the same time, the buffering and tracking of event time is also costlier than operating
-	 * with processing time, and typically also introduces more latency. The amount of extra
-	 * cost depends mostly on how much out of order the elements arrive, i.e., how long the time span
-	 * between the arrival of early and late elements is. With respect to the "time watermarks", this
-	 * means that the cost typically depends on how early or late the watermarks can be generated
-	 * for their timestamp.
-	 * <p>
-	 * In relation to {@link #IngestionTime}, the event time is similar, but refers the the event's
-	 * original time, rather than the time assigned at the data source. Practically, that means that
-	 * event time has generally more meaning, but also that it takes longer to determine that all
-	 * elements for a certain time have arrived.
-	 */
-	EventTime
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java
deleted file mode 100644
index c2d2182..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.checkpoint;
-
-/**
- * This interface must be implemented by functions/operations that want to receive
- * a commit notification once a checkpoint has been completely acknowledged by all
- * participants.
- */
-public interface CheckpointNotifier {
-
-	/**
-	 * This method is called as a notification once a distributed checkpoint has been completed.
-	 * 
-	 * Note that any exception during this method will not cause the checkpoint to
-	 * fail any more.
-	 * 
-	 * @param checkpointId The ID of the checkpoint that has been completed.
-	 * @throws Exception
-	 */
-	void notifyCheckpointComplete(long checkpointId) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
deleted file mode 100644
index ac1cbfb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.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.checkpoint;
-
-import java.io.Serializable;
-
-/**
- * This method must be implemented by functions that have state that needs to be
- * checkpointed. The functions get a call whenever a checkpoint should take place
- * and return a snapshot of their state, which will be checkpointed.
- * 
- * <p>This interface marks a function as <i>synchronously</i> checkpointed. While the
- * state is written, the function is not called, so the function needs not return a
- * copy of its state, but may return a reference to its state. Functions that can
- * continue to work and mutate the state, even while the state snapshot is being accessed,
- * can implement the {@link org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously}
- * interface.</p>
- * 
- * @param <T> The type of the operator state.
- */
-public interface Checkpointed<T extends Serializable> {
-
-	/**
-	 * Gets the current state of the function of operator. The state must reflect the result of all
-	 * prior invocations to this function. 
-	 * 
-	 * @param checkpointId The ID of the checkpoint.
-	 * @param checkpointTimestamp The timestamp of the checkpoint, as derived by
-	 *                            System.currentTimeMillis() on the JobManager.
-	 *                            
-	 * @return A snapshot of the operator state.
-	 * 
-	 * @throws Exception Thrown if the creation of the state object failed. This causes the
-	 *                   checkpoint to fail. The system may decide to fail the operation (and trigger
-	 *                   recovery), or to discard this checkpoint attempt and to continue running
-	 *                   and to try again with the next checkpoint attempt.
-	 */
-	T snapshotState(long checkpointId, long checkpointTimestamp) throws Exception;
-
-	/**
-	 * Restores the state of the function or operator to that of a previous checkpoint.
-	 * This method is invoked when a function is executed as part of a recovery run.
-	 *
-	 * Note that restoreState() is called before open().
-	 *
-	 * @param state The state to be restored. 
-	 */
-	void restoreState(T state) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
deleted file mode 100644
index 4bd89c4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.checkpoint;
-
-import java.io.Serializable;
-
-/**
- * This interface marks a function/operator as <i>asynchronously checkpointed</i>.
- * Similar to the {@link Checkpointed} interface, the function must produce a
- * snapshot of its state. However, the function must be able to continue working
- * and mutating its state without mutating the returned state snapshot.
- * 
- * <p>Asynchronous checkpoints are desirable, because they allow the data streams at the
- * point of the checkpointed function/operator to continue running while the checkpoint
- * is in progress.</p>
- * 
- * <p>To be able to support asynchronous snapshots, the state returned by the
- * {@link #snapshotState(long, long)} method is typically a copy or shadow copy
- * of the actual state.</p>
- */
-public interface CheckpointedAsynchronously<T extends Serializable> extends Checkpointed<T> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
deleted file mode 100644
index 7034b11..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.collector.selector;
-
-import java.util.ArrayList;
-
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-public class BroadcastOutputSelectorWrapper<OUT> implements OutputSelectorWrapper<OUT> {
-
-	private static final long serialVersionUID = 1L;
-	
-	private final ArrayList<Collector<StreamRecord<OUT>>> outputs;
-
-	public BroadcastOutputSelectorWrapper() {
-		outputs = new ArrayList<Collector<StreamRecord<OUT>>>();
-	}
-	
-	@Override
-	public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge) {
-		outputs.add(output);
-	}
-
-	@Override
-	public Iterable<Collector<StreamRecord<OUT>>> getSelectedOutputs(OUT record) {
-		return outputs;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
deleted file mode 100644
index 84558fc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
+++ /dev/null
@@ -1,97 +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.collector.selector;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class DirectedOutputSelectorWrapper<OUT> implements OutputSelectorWrapper<OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(DirectedOutputSelectorWrapper.class);
-
-	private List<OutputSelector<OUT>> outputSelectors;
-
-	private HashMap<String, ArrayList<Collector<StreamRecord<OUT>>>> outputMap;
-	private HashSet<Collector<StreamRecord<OUT>>> selectAllOutputs;
-
-	public DirectedOutputSelectorWrapper(List<OutputSelector<OUT>> outputSelectors) {
-		this.outputSelectors = outputSelectors;
-		this.selectAllOutputs = new HashSet<Collector<StreamRecord<OUT>>>();
-		this.outputMap = new HashMap<String, ArrayList<Collector<StreamRecord<OUT>>>>();
-	}
-	
-	@Override
-	public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge) {
-		List<String> selectedNames = edge.getSelectedNames();
-
-		if (selectedNames.isEmpty()) {
-			selectAllOutputs.add(output);
-		}
-		else {
-			for (String selectedName : selectedNames) {
-				if (!outputMap.containsKey(selectedName)) {
-					outputMap.put(selectedName, new ArrayList<Collector<StreamRecord<OUT>>>());
-					outputMap.get(selectedName).add(output);
-				}
-				else {
-					if (!outputMap.get(selectedName).contains(output)) {
-						outputMap.get(selectedName).add(output);
-					}
-				}
-			}
-		}
-	}
-
-	@Override
-	public Iterable<Collector<StreamRecord<OUT>>> getSelectedOutputs(OUT record) {
-		Set<Collector<StreamRecord<OUT>>> selectedOutputs = new HashSet<Collector<StreamRecord<OUT>>>(selectAllOutputs);
-
-		for (OutputSelector<OUT> outputSelector : outputSelectors) {
-			Iterable<String> outputNames = outputSelector.select(record);
-
-			for (String outputName : outputNames) {
-				List<Collector<StreamRecord<OUT>>> outputList = outputMap.get(outputName);
-
-				try {
-					selectedOutputs.addAll(outputList);
-				} catch (NullPointerException e) {
-					if (LOG.isErrorEnabled()) {
-						String format = String.format(
-								"Cannot emit because no output is selected with the name: %s",
-								outputName);
-						LOG.error(format);
-					}
-				}
-			}
-		}
-
-		return selectedOutputs;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
deleted file mode 100644
index 9c6eede..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.collector.selector;
-
-import java.io.Serializable;
-
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-
-/**
- * Interface for defining an OutputSelector for a {@link SplitStream} using
- * the {@link SingleOutputStreamOperator#split} call. Every output object of a
- * {@link SplitStream} will run through this operator to select outputs.
- * 
- * @param <OUT>
- *            Type parameter of the split values.
- */
-public interface OutputSelector<OUT> extends Serializable {
-	/**
-	 * Method for selecting output names for the emitted objects when using the
-	 * {@link SingleOutputStreamOperator#split} method. The values will be
-	 * emitted only to output names which are contained in the returned
-	 * iterable.
-	 * 
-	 * @param value
-	 *            Output object for which the output selection should be made.
-	 */
-	public Iterable<String> select(OUT value);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
deleted file mode 100644
index f25c995..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.collector.selector;
-
-import java.io.Serializable;
-
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-public interface OutputSelectorWrapper<OUT> extends Serializable {
-
-	public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge);
-
-	public Iterable<Collector<StreamRecord<OUT>>> getSelectedOutputs(OUT record);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java
deleted file mode 100644
index dca2ede..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.collector.selector;
-
-import java.util.List;
-
-public class OutputSelectorWrapperFactory {
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	public static OutputSelectorWrapper<?> create(List<OutputSelector<?>> outputSelectors) {
-		if (outputSelectors.size() == 0) {
-			return new BroadcastOutputSelectorWrapper();
-		} else {
-			return new DirectedOutputSelectorWrapper(outputSelectors);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
deleted file mode 100644
index 7191304..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ /dev/null
@@ -1,556 +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.FoldFunction;
-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.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
-import org.apache.flink.streaming.api.functions.windowing.FoldAllWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-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 AllWindowedStream} 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 AllWindowedStream} is purely and API construct, during runtime
- * the {@code AllWindowedStream} 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 AllWindowedStream<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 AllWindowedStream(DataStream<T> input,
-			WindowAssigner<? super T, W> windowAssigner) {
-		this.input = input;
-		this.windowAssigner = windowAssigner;
-		this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
-	}
-
-	/**
-	 * Sets the {@code Trigger} that should be used to trigger window emission.
-	 */
-	public AllWindowedStream<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 AllWindowedStream<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 SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> function) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "Reduce at " + callLocation;
-
-		SingleOutputStreamOperator<T, ?> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
-		if (result != null) {
-			return result;
-		}
-
-		String opName = "NonParallelTriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-
-		OneInputStreamOperator<T, T> operator;
-
-		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
-
-		if (evictor != null) {
-			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					new ReduceAllWindowFunction<W, T>(function),
-					trigger,
-					evictor).enableSetProcessingTime(setProcessingTime);
-
-		} 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,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
-					new ReduceAllWindowFunction<W, T>(function),
-					trigger).enableSetProcessingTime(setProcessingTime);
-		}
-
-		return input.transform(opName, input.getType(), operator).setParallelism(1);
-	}
-
-	/**
-	 * Applies the given fold function to each 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.
-	 *
-	 * @param function The fold function.
-	 * @return The data stream that is the result of applying the fold function to the window.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-
-		TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
-				Utils.getCallLocationName(), true);
-
-		return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
-	}
-
-	/**
-	 * Applies the given fold function to each 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.
-	 *
-	 * @param function The fold function.
-	 * @return The data stream that is the result of applying the fold function to the window.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-		return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
-	}
-
-	/**
-	 * 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> SingleOutputStreamOperator<R, ?> apply(AllWindowFunction<T, R, W> function) {
-		TypeInformation<T> inType = input.getType();
-		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
-				function, AllWindowFunction.class, true, true, inType, null, false);
-
-		return apply(function, resultType);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each evaluation
-	 * of the window for each key individually. The output of the window function is interpreted
-	 * as a regular non-windowed stream.
-	 * <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> SingleOutputStreamOperator<R, ?> apply(AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
-		//clean the closure
-		function = input.getExecutionEnvironment().clean(function);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "WindowApply at " + callLocation;
-
-		SingleOutputStreamOperator<R, ?> result = createFastTimeOperatorIfValid(function, resultType, udfName);
-		if (result != null) {
-			return result;
-		}
-
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-
-		NonKeyedWindowOperator<T, R, W> operator;
-
-		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
-
-		if (evictor != null) {
-			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger,
-					evictor).enableSetProcessingTime(setProcessingTime);
-
-		} else {
-			operator = new NonKeyedWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger).enableSetProcessingTime(setProcessingTime);
-		}
-
-		return input.transform(opName, resultType, operator).setParallelism(1);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <p>
-	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-	 *
-	 * @param preAggregator The reduce function that is used for 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> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, AllWindowFunction<T, R, W> function) {
-		TypeInformation<T> inType = input.getType();
-		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
-				function, AllWindowFunction.class, true, true, inType, null, false);
-
-		return apply(preAggregator, function, resultType);
-	}
-
-	/**
-	 * Applies the given window function to each window. The window function is called for each
-	 * evaluation of the window for each key individually. The output of the window function is
-	 * interpreted as a regular non-windowed stream.
-	 *
-	 * <p>
-	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-	 *
-	 * @param preAggregator The reduce function that is used for pre-aggregation
-	 * @param function The window function.
-	 * @param resultType Type information for the result type of the window function
-	 * @return The data stream that is the result of applying the window function to the window.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
-		//clean the closures
-		function = input.getExecutionEnvironment().clean(function);
-		preAggregator = input.getExecutionEnvironment().clean(preAggregator);
-
-		String callLocation = Utils.getCallLocationName();
-		String udfName = "WindowApply at " + callLocation;
-
-		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
-
-		OneInputStreamOperator<T, R> operator;
-
-		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
-
-		if (evictor != null) {
-			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					new HeapWindowBuffer.Factory<T>(),
-					function,
-					trigger,
-					evictor).enableSetProcessingTime(setProcessingTime);
-
-		} else {
-			operator = new NonKeyedWindowOperator<>(windowAssigner,
-					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
-					new PreAggregatingHeapWindowBuffer.Factory<>(preAggregator),
-					function,
-					trigger).enableSetProcessingTime(setProcessingTime);
-		}
-
-		return input.transform(opName, resultType, operator).setParallelism(1);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Aggregations on the  windows
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Applies an aggregation that sums every window of the data stream at the
-	 * given position.
-	 *
-	 * @param positionToSum The position in the tuple/array to sum
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
-		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that sums every window of the pojo data stream at
-	 * the given field for every window.
-	 *
-	 * <p>
-	 * A field expression is either
-	 * the name of a public field or a getter method with parentheses of the
-	 * stream's underlying type. A dot can be used to drill down into objects,
-	 * as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field to sum
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(String field) {
-		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum value of every window
-	 * of the data stream at the given position.
-	 *
-	 * @param positionToMin The position to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
-		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum value of the pojo data
-	 * stream at the given field expression for every window.
-	 *
-	 * <p>
-	 * A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(String field) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns the first element by default.
-	 *
-	 * @param positionToMinBy
-	 *            The position to minimize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns the first element by default.
-	 *
-	 * @param positionToMinBy The position to minimize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the minimum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * minimum value the operator returns either the first or last one depending
-	 * on the parameter setting.
-	 *
-	 * @param positionToMinBy The position to minimize
-	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
-		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the minimum element of the pojo
-	 * data stream by the given field expression for every window. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream DataStreams} underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @param first If True then in case of field equality the first object will be returned
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum value of every window of
-	 * the data stream at the given position.
-	 *
-	 * @param positionToMax The position to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
-		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the maximum value of the pojo data
-	 * stream at the given field expression for every window. A field expression
-	 * is either the name of a public field or a getter method with parentheses
-	 * of the {@link DataStream DataStreams} underlying type. A dot can be used to drill
-	 * down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(String field) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns the first by default.
-	 *
-	 * @param positionToMaxBy
-	 *            The position to maximize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns the first by default.
-	 *
-	 * @param positionToMaxBy
-	 *            The position to maximize by
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that gives the maximum element of every window of
-	 * the data stream by the given position. If more elements have the same
-	 * maximum value the operator returns either the first or last one depending
-	 * on the parameter setting.
-	 *
-	 * @param positionToMaxBy The position to maximize by
-	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
-		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the maximum element of the pojo
-	 * data stream by the given field expression for every window. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field The field expression based on which the aggregation will be applied.
-	 * @param first If True then in case of field equality the first object will be returned
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
-	}
-
-	private SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregator) {
-		return reduce(aggregator);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-
-	private <R> SingleOutputStreamOperator<R, ?> createFastTimeOperatorIfValid(
-			Function function,
-			TypeInformation<R> resultType,
-			String functionName) {
-
-		// TODO: add once non-parallel fast aligned time windows operator is ready
-		return null;
-	}
-
-	public StreamExecutionEnvironment getExecutionEnvironment() {
-		return input.getExecutionEnvironment();
-	}
-
-	public TypeInformation<T> getInputType() {
-		return input.getType();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
deleted file mode 100644
index d1da783..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
+++ /dev/null
@@ -1,575 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.translation.WrappingFunction;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-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.util.Collector;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- *{@code CoGroupedStreams} represents two {@link DataStream DataStreams} that have been co-grouped.
- * A streaming co-group operation is evaluated over elements in a window.
- *
- * <p>
- * To finalize co-group operation you also need to specify a {@link KeySelector} for
- * both the first and second input and a {@link WindowAssigner}.
- *
- * <p>
- * Note: Right now, the groups are being built in memory so you need to ensure that they don't
- * get too big. Otherwise the JVM might crash.
- *
- * <p>
- * Example:
- *
- * <pre> {@code
- * DataStream<Tuple2<String, Integer>> one = ...;
- * DataStream<Tuple2<String, Integer>> two = ...;
- *
- * DataStream<T> result = one.coGroup(two)
- *     .where(new MyFirstKeySelector())
- *     .equalTo(new MyFirstKeySelector())
- *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
- *     .apply(new MyCoGroupFunction());
- * } </pre>
- */
-public class CoGroupedStreams<T1, T2> {
-
-	/** The first input stream */
-	private final DataStream<T1> input1;
-
-	/** The second input stream */
-	private final DataStream<T2> input2;
-
-	/**
-	 * Creates new CoGroped data streams, which are the first step towards building a streaming co-group.
-	 * 
-	 * @param input1 The first data stream.
-	 * @param input2 The second data stream.
-	 */
-	public CoGroupedStreams(DataStream<T1> input1, DataStream<T2> input2) {
-		this.input1 = requireNonNull(input1);
-		this.input2 = requireNonNull(input2);
-	}
-
-	/**
-	 * Specifies a {@link KeySelector} for elements from the first input.
-	 */
-	public <KEY> Where<KEY> where(KeySelector<T1, KEY> keySelector)  {
-		TypeInformation<KEY> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
-		return new Where<>(input1.clean(keySelector), keyType);
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * CoGrouped streams that have the key for one side defined.
-	 * 
-	 * @param <KEY> The type of the key.
-	 */
-	public class Where<KEY> {
-
-		private final KeySelector<T1, KEY> keySelector1;
-		private final TypeInformation<KEY> keyType;
-
-		Where(KeySelector<T1, KEY> keySelector1, TypeInformation<KEY> keyType) {
-			this.keySelector1 = keySelector1;
-			this.keyType = keyType;
-		}
-	
-		/**
-		 * Specifies a {@link KeySelector} for elements from the second input.
-		 */
-		public EqualTo equalTo(KeySelector<T2, KEY> keySelector)  {
-			TypeInformation<KEY> otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
-			if (!otherKey.equals(this.keyType)) {
-				throw new IllegalArgumentException("The keys for the two inputs are not equal: " + 
-						"first key = " + this.keyType + " , second key = " + otherKey);
-			}
-			
-			return new EqualTo(input2.clean(keySelector));
-		}
-
-		// --------------------------------------------------------------------
-		
-		/**
-		 * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs.
-		 */
-		public class EqualTo {
-
-			private final KeySelector<T2, KEY> keySelector2;
-
-			EqualTo(KeySelector<T2, KEY> keySelector2) {
-				this.keySelector2 = requireNonNull(keySelector2);
-			}
-
-			/**
-			 * Specifies the window on which the co-group operation works.
-			 */
-			public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
-				return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs as
-	 * well as a {@link WindowAssigner}.
-	 *
-	 * @param <T1> Type of the elements from the first input
-	 * @param <T2> Type of the elements from the second input
-	 * @param <KEY> Type of the key. This must be the same for both inputs
-	 * @param <W> Type of {@link Window} on which the co-group operation works.
-	 */
-	public static class WithWindow<T1, T2, KEY, W extends Window> {
-		private final DataStream<T1> input1;
-		private final DataStream<T2> input2;
-
-		private final KeySelector<T1, KEY> keySelector1;
-		private final KeySelector<T2, KEY> keySelector2;
-		
-		private final TypeInformation<KEY> keyType;
-
-		private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
-
-		private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
-
-		private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
-
-		protected WithWindow(DataStream<T1> input1,
-				DataStream<T2> input2,
-				KeySelector<T1, KEY> keySelector1,
-				KeySelector<T2, KEY> keySelector2,
-				TypeInformation<KEY> keyType,
-				WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
-				Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
-				Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
-			this.input1 = input1;
-			this.input2 = input2;
-
-			this.keySelector1 = keySelector1;
-			this.keySelector2 = keySelector2;
-			this.keyType = keyType;
-			
-			this.windowAssigner = windowAssigner;
-			this.trigger = trigger;
-			this.evictor = evictor;
-		}
-
-		/**
-		 * Sets the {@code Trigger} that should be used to trigger window emission.
-		 */
-		public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
-			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
-					windowAssigner, newTrigger, evictor);
-		}
-
-		/**
-		 * 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 WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
-			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
-					windowAssigner, trigger, newEvictor);
-		}
-
-		/**
-		 * Completes the co-group operation with the user function that is executed
-		 * for windowed groups.
-		 */
-		public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function) {
-
-			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
-					function,
-					CoGroupFunction.class,
-					true,
-					true,
-					input1.getType(),
-					input2.getType(),
-					"CoGroup",
-					false);
-
-			return apply(function, resultType);
-		}
-
-		/**
-		 * Completes the co-group operation with the user function that is executed
-		 * for windowed groups.
-		 */
-		public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function, TypeInformation<T> resultType) {
-			//clean the closure
-			function = input1.getExecutionEnvironment().clean(function);
-
-			UnionTypeInfo<T1, T2> unionType = new UnionTypeInfo<>(input1.getType(), input2.getType());
-			UnionKeySelector<T1, T2, KEY> unionKeySelector = new UnionKeySelector<>(keySelector1, keySelector2);
-			
-			DataStream<TaggedUnion<T1, T2>> taggedInput1 = input1
-					.map(new Input1Tagger<T1, T2>())
-					.returns(unionType);
-			DataStream<TaggedUnion<T1, T2>> taggedInput2 = input2
-					.map(new Input2Tagger<T1, T2>())
-					.returns(unionType);
-
-			DataStream<TaggedUnion<T1, T2>> unionStream = taggedInput1.union(taggedInput2);
-			
-			// we explicitly create the keyed stream to manually pass the key type information in
-			WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp = 
-					new KeyedStream<TaggedUnion<T1, T2>, KEY>(unionStream, unionKeySelector, keyType)
-					.window(windowAssigner);
-
-			if (trigger != null) {
-				windowOp.trigger(trigger);
-			}
-			if (evictor != null) {
-				windowOp.evictor(evictor);
-			}
-
-			return windowOp.apply(new CoGroupWindowFunction<T1, T2, T, KEY, W>(function), resultType);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Data type and type information for Tagged Union
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Internal class for implementing tagged union co-group.
-	 */
-	public static class TaggedUnion<T1, T2> {
-		private final T1 one;
-		private final T2 two;
-
-		private TaggedUnion(T1 one, T2 two) {
-			this.one = one;
-			this.two = two;
-		}
-
-		public boolean isOne() {
-			return one != null;
-		}
-
-		public boolean isTwo() {
-			return two != null;
-		}
-
-		public T1 getOne() {
-			return one;
-		}
-
-		public T2 getTwo() {
-			return two;
-		}
-
-		public static <T1, T2> TaggedUnion<T1, T2> one(T1 one) {
-			return new TaggedUnion<>(one, null);
-		}
-
-		public static <T1, T2> TaggedUnion<T1, T2> two(T2 two) {
-			return new TaggedUnion<>(null, two);
-		}
-	}
-
-	private static class UnionTypeInfo<T1, T2> extends TypeInformation<TaggedUnion<T1, T2>> {
-		private static final long serialVersionUID = 1L;
-
-		TypeInformation<T1> oneType;
-		TypeInformation<T2> twoType;
-
-		public UnionTypeInfo(TypeInformation<T1> oneType,
-				TypeInformation<T2> twoType) {
-			this.oneType = oneType;
-			this.twoType = twoType;
-		}
-
-		@Override
-		public boolean isBasicType() {
-			return false;
-		}
-
-		@Override
-		public boolean isTupleType() {
-			return false;
-		}
-
-		@Override
-		public int getArity() {
-			return 2;
-		}
-
-		@Override
-		public int getTotalFields() {
-			return 2;
-		}
-
-		@Override
-		@SuppressWarnings("unchecked, rawtypes")
-		public Class<TaggedUnion<T1, T2>> getTypeClass() {
-			return (Class) TaggedUnion.class;
-		}
-
-		@Override
-		public boolean isKeyType() {
-			return true;
-		}
-
-		@Override
-		public TypeSerializer<TaggedUnion<T1, T2>> createSerializer(ExecutionConfig config) {
-			return new UnionSerializer<>(oneType.createSerializer(config), twoType.createSerializer(config));
-		}
-
-		@Override
-		public String toString() {
-			return "TaggedUnion<" + oneType + ", " + twoType + ">";
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj instanceof UnionTypeInfo) {
-				@SuppressWarnings("unchecked")
-				UnionTypeInfo<T1, T2> unionTypeInfo = (UnionTypeInfo<T1, T2>) obj;
-
-				return unionTypeInfo.canEqual(this) && oneType.equals(unionTypeInfo.oneType) && twoType.equals(unionTypeInfo.twoType);
-			} else {
-				return false;
-			}
-		}
-
-		@Override
-		public int hashCode() {
-			return 31 *  oneType.hashCode() + twoType.hashCode();
-		}
-
-		@Override
-		public boolean canEqual(Object obj) {
-			return obj instanceof UnionTypeInfo;
-		}
-	}
-
-	private static class UnionSerializer<T1, T2> extends TypeSerializer<TaggedUnion<T1, T2>> {
-		private static final long serialVersionUID = 1L;
-
-		private final TypeSerializer<T1> oneSerializer;
-		private final TypeSerializer<T2> twoSerializer;
-
-		public UnionSerializer(TypeSerializer<T1> oneSerializer,
-				TypeSerializer<T2> twoSerializer) {
-			this.oneSerializer = oneSerializer;
-			this.twoSerializer = twoSerializer;
-		}
-
-		@Override
-		public boolean isImmutableType() {
-			return false;
-		}
-
-		@Override
-		public TypeSerializer<TaggedUnion<T1, T2>> duplicate() {
-			return this;
-		}
-
-		@Override
-		public TaggedUnion<T1, T2> createInstance() {
-			return null;
-		}
-
-		@Override
-		public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from) {
-			if (from.isOne()) {
-				return TaggedUnion.one(oneSerializer.copy(from.getOne()));
-			} else {
-				return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
-			}
-		}
-
-		@Override
-		public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from, TaggedUnion<T1, T2> reuse) {
-			if (from.isOne()) {
-				return TaggedUnion.one(oneSerializer.copy(from.getOne()));
-			} else {
-				return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
-			}		}
-
-		@Override
-		public int getLength() {
-			return -1;
-		}
-
-		@Override
-		public void serialize(TaggedUnion<T1, T2> record, DataOutputView target) throws IOException {
-			if (record.isOne()) {
-				target.writeByte(1);
-				oneSerializer.serialize(record.getOne(), target);
-			} else {
-				target.writeByte(2);
-				twoSerializer.serialize(record.getTwo(), target);
-			}
-		}
-
-		@Override
-		public TaggedUnion<T1, T2> deserialize(DataInputView source) throws IOException {
-			byte tag = source.readByte();
-			if (tag == 1) {
-				return TaggedUnion.one(oneSerializer.deserialize(source));
-			} else {
-				return TaggedUnion.two(twoSerializer.deserialize(source));
-			}
-		}
-
-		@Override
-		public TaggedUnion<T1, T2> deserialize(TaggedUnion<T1, T2> reuse,
-				DataInputView source) throws IOException {
-			byte tag = source.readByte();
-			if (tag == 1) {
-				return TaggedUnion.one(oneSerializer.deserialize(source));
-			} else {
-				return TaggedUnion.two(twoSerializer.deserialize(source));
-			}
-		}
-
-		@Override
-		public void copy(DataInputView source, DataOutputView target) throws IOException {
-			byte tag = source.readByte();
-			target.writeByte(tag);
-			if (tag == 1) {
-				oneSerializer.copy(source, target);
-			} else {
-				twoSerializer.copy(source, target);
-			}
-		}
-
-		@Override
-		public int hashCode() {
-			return 31 * oneSerializer.hashCode() + twoSerializer.hashCode();
-		}
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public boolean equals(Object obj) {
-			if (obj instanceof UnionSerializer) {
-				UnionSerializer<T1, T2> other = (UnionSerializer<T1, T2>) obj;
-
-				return other.canEqual(this) && oneSerializer.equals(other.oneSerializer) && twoSerializer.equals(other.twoSerializer);
-			} else {
-				return false;
-			}
-		}
-
-		@Override
-		public boolean canEqual(Object obj) {
-			return obj instanceof UnionSerializer;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utility functions that implement the CoGroup logic based on the tagged
-	//  untion window reduce
-	// ------------------------------------------------------------------------
-	
-	private static class Input1Tagger<T1, T2> implements MapFunction<T1, TaggedUnion<T1, T2>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public TaggedUnion<T1, T2> map(T1 value) throws Exception {
-			return TaggedUnion.one(value);
-		}
-	}
-
-	private static class Input2Tagger<T1, T2> implements MapFunction<T2, TaggedUnion<T1, T2>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public TaggedUnion<T1, T2> map(T2 value) throws Exception {
-			return TaggedUnion.two(value);
-		}
-	}
-
-	private static class UnionKeySelector<T1, T2, KEY> implements KeySelector<TaggedUnion<T1, T2>, KEY> {
-		private static final long serialVersionUID = 1L;
-
-		private final KeySelector<T1, KEY> keySelector1;
-		private final KeySelector<T2, KEY> keySelector2;
-
-		public UnionKeySelector(KeySelector<T1, KEY> keySelector1,
-				KeySelector<T2, KEY> keySelector2) {
-			this.keySelector1 = keySelector1;
-			this.keySelector2 = keySelector2;
-		}
-
-		@Override
-		public KEY getKey(TaggedUnion<T1, T2> value) throws Exception{
-			if (value.isOne()) {
-				return keySelector1.getKey(value.getOne());
-			} else {
-				return keySelector2.getKey(value.getTwo());
-			}
-		}
-	}
-
-	private static class CoGroupWindowFunction<T1, T2, T, KEY, W extends Window>
-			extends WrappingFunction<CoGroupFunction<T1, T2, T>>
-			implements WindowFunction<TaggedUnion<T1, T2>, T, KEY, W> {
-		
-		private static final long serialVersionUID = 1L;
-
-		public CoGroupWindowFunction(CoGroupFunction<T1, T2, T> userFunction) {
-			super(userFunction);
-		}
-
-		@Override
-		public void apply(KEY key,
-				W window,
-				Iterable<TaggedUnion<T1, T2>> values,
-				Collector<T> out) throws Exception {
-			
-			List<T1> oneValues = new ArrayList<>();
-			List<T2> twoValues = new ArrayList<>();
-			
-			for (TaggedUnion<T1, T2> val: values) {
-				if (val.isOne()) {
-					oneValues.add(val.getOne());
-				} else {
-					twoValues.add(val.getTwo());
-				}
-			}
-			wrappedFunction.coGroup(oneValues, twoValues, out);
-		}
-	}
-}


[34/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
deleted file mode 100644
index 3e1ff57..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This transformation represents a union of several input
- * {@link StreamTransformation StreamTransformations}.
- *
- * <p>
- * This does not create a physical operation, it only affects how upstream operations are
- * connected to downstream operations.
- *
- * @param <T> The type of the elements that result from this {@code UnionTransformation}
- */
-public class UnionTransformation<T> extends StreamTransformation<T> {
-	private final List<StreamTransformation<T>> inputs;
-
-	/**
-	 * Creates a new {@code UnionTransformation} from the given input {@code StreamTransformations}.
-	 *
-	 * <p>
-	 * The input {@code StreamTransformations} must all have the same type.
-	 *
-	 * @param inputs The list of input {@code StreamTransformations}
-	 */
-	public UnionTransformation(List<StreamTransformation<T>> inputs) {
-		super("Union", inputs.get(0).getOutputType(), inputs.get(0).getParallelism());
-
-		for (StreamTransformation<T> input: inputs) {
-			if (!input.getOutputType().equals(getOutputType())) {
-				throw new UnsupportedOperationException("Type mismatch in input " + input);
-			}
-		}
-
-		this.inputs = Lists.newArrayList(inputs);
-	}
-
-	/**
-	 * Returns the list of input {@code StreamTransformations}.
-	 */
-	public List<StreamTransformation<T>> getInputs() {
-		return inputs;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		for (StreamTransformation<T> input: inputs) {
-			result.addAll(input.getTransitivePredecessors());
-		}
-		return result;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation.");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
deleted file mode 100644
index 838c24a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
+++ /dev/null
@@ -1,76 +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.watermark;
-
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-
-/**
- * A Watermark tells operators that receive it that no elements with a timestamp older or equal
- * to the watermark timestamp should arrive at the operator. Watermarks are emitted at the
- * sources and propagate through the operators of the topology. Operators must themselves emit
- * watermarks to downstream operators using
- * {@link org.apache.flink.streaming.api.operators.Output#emitWatermark(Watermark)}. Operators that
- * do not internally buffer elements can always forward the watermark that they receive. Operators
- * that buffer elements, such as window operators, must forward a watermark after emission of
- * elements that is triggered by the arriving watermark.
- *
- * <p>
- * In some cases a watermark is only a heuristic and operators should be able to deal with
- * late elements. They can either discard those or update the result and emit updates/retractions
- * to downstream operations.
- *
- * <p>
- * When a source closes it will emit a final watermark with timestamp {@code Long.MAX_VALUE}. When
- * an operator receives this it will know that no more input will be arriving in the future.
- *
- */
-public class Watermark extends StreamElement {
-
-	/** The timestamp of the watermark */
-	private final long timestamp;
-
-	/**
-	 * Creates a new watermark with the given timestamp.
-	 */
-	public Watermark(long timestamp) {
-		this.timestamp = timestamp;
-	}
-
-	/**
-	 * Returns the timestamp associated with this {@link Watermark} in milliseconds.
-	 */
-	public long getTimestamp() {
-		return timestamp;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		return this == o ||
-				o != null && o.getClass() == Watermark.class && ((Watermark) o).timestamp == this.timestamp;
-	}
-
-	@Override
-	public int hashCode() {
-		return (int) (timestamp ^ (timestamp >>> 32));
-	}
-
-	@Override
-	public String toString() {
-		return "Watermark @ " + timestamp;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
deleted file mode 100644
index 4d5b9d7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.assigners;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * A {@link WindowAssigner} that assigns all elements to the same global window.
- *
- * <p>
- * Use this if you want to use a {@link Trigger} and
- * {@link org.apache.flink.streaming.api.windowing.evictors.Evictor} to do flexible, policy based
- * windows.
- */
-public class GlobalWindows extends WindowAssigner<Object, GlobalWindow> {
-	private static final long serialVersionUID = 1L;
-
-	private GlobalWindows() {}
-
-	@Override
-	public Collection<GlobalWindow> assignWindows(Object element, long timestamp) {
-		return Collections.singletonList(GlobalWindow.get());
-	}
-
-	@Override
-	public Trigger<Object, GlobalWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
-		return new NeverTrigger();
-	}
-
-	@Override
-	public String toString() {
-		return "GlobalWindows()";
-	}
-
-	/**
-	 * Creates a new {@code GlobalWindows} {@link WindowAssigner} that assigns
-	 * all elements to the same {@link GlobalWindow}.
-	 *
-	 * @return The global window policy.
-	 */
-	public static GlobalWindows create() {
-		return new GlobalWindows();
-	}
-
-	/**
-	 * A trigger that never fires, as default Trigger for GlobalWindows.
-	 */
-	private static class NeverTrigger implements Trigger<Object, GlobalWindow> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public TriggerResult onElement(Object element,
-				long timestamp,
-				GlobalWindow window,
-				TriggerContext ctx) {
-				return TriggerResult.CONTINUE;
-		}
-
-		@Override
-		public TriggerResult onEventTime(long time, TriggerContext ctx) {
-			return TriggerResult.CONTINUE;
-		}
-
-		@Override
-		public TriggerResult onProcessingTime(long time, TriggerContext ctx) {
-			return TriggerResult.CONTINUE;
-		}
-	}
-
-	@Override
-	public TypeSerializer<GlobalWindow> getWindowSerializer(ExecutionConfig executionConfig) {
-		return new GlobalWindow.Serializer();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
deleted file mode 100644
index 5f7ab45..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.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.assigners;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-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.triggers.EventTimeTrigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * A {@link WindowAssigner} that windows elements into sliding windows based on the timestamp of the
- * elements. Windows can possibly overlap.
- *
- * <p>
- * For example, in order to window into windows of 1 minute, every 10 seconds:
- * <pre> {@code
- * DataStream<Tuple2<String, Integer>> in = ...;
- * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
- * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
- *   keyed.window(SlidingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
- * } </pre>
- */
-public class SlidingTimeWindows extends WindowAssigner<Object, TimeWindow> {
-	private static final long serialVersionUID = 1L;
-
-	private final long size;
-
-	private final long slide;
-
-	private SlidingTimeWindows(long size, long slide) {
-		this.size = size;
-		this.slide = slide;
-	}
-
-	@Override
-	public Collection<TimeWindow> assignWindows(Object element, long timestamp) {
-		List<TimeWindow> windows = new ArrayList<>((int) (size / slide));
-		long lastStart = timestamp - timestamp % slide;
-		for (long start = lastStart;
-			start > timestamp - size;
-			start -= slide) {
-			windows.add(new TimeWindow(start, start + size));
-		}
-		return windows;
-	}
-
-	public long getSize() {
-		return size;
-	}
-
-	public long getSlide() {
-		return slide;
-	}
-
-	@Override
-	public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
-		if (env.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
-			return ProcessingTimeTrigger.create();
-		} else {
-			return EventTimeTrigger.create();
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "SlidingTimeWindows(" + size + ", " + slide + ")";
-	}
-
-	/**
-	 * Creates a new {@code SlidingTimeWindows} {@link WindowAssigner} that assigns
-	 * elements to sliding time windows based on the element timestamp.
-	 *
-	 * @param size The size of the generated windows.
-	 * @param slide The slide interval of the generated windows.
-	 * @return The time policy.
-	 */
-	public static SlidingTimeWindows of(AbstractTime size, AbstractTime slide) {
-		return new SlidingTimeWindows(size.toMilliseconds(), slide.toMilliseconds());
-	}
-
-	@Override
-	public TypeSerializer<TimeWindow> getWindowSerializer(ExecutionConfig executionConfig) {
-		return new TimeWindow.Serializer();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
deleted file mode 100644
index 463b2c4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.assigners;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-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.triggers.EventTimeTrigger;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * A {@link WindowAssigner} that windows elements into windows based on the timestamp of the
- * elements. Windows cannot overlap.
- *
- * <p>
- * For example, in order to window into windows of 1 minute, every 10 seconds:
- * <pre> {@code
- * DataStream<Tuple2<String, Integer>> in = ...;
- * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
- * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
- *   keyed.window(TumblingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
- * } </pre>
- */
-public class TumblingTimeWindows extends WindowAssigner<Object, TimeWindow> {
-	private static final long serialVersionUID = 1L;
-
-	private long size;
-
-	private TumblingTimeWindows(long size) {
-		this.size = size;
-	}
-
-	@Override
-	public Collection<TimeWindow> assignWindows(Object element, long timestamp) {
-		long start = timestamp - (timestamp % size);
-		return Collections.singletonList(new TimeWindow(start, start + size));
-	}
-
-	public long getSize() {
-		return size;
-	}
-
-	@Override
-	public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
-		if (env.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
-			return ProcessingTimeTrigger.create();
-		} else {
-			return EventTimeTrigger.create();
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "TumblingTimeWindows(" + size + ")";
-	}
-
-	/**
-	 * Creates a new {@code TumblingTimeWindows} {@link WindowAssigner} that assigns
-	 * elements to time windows based on the element timestamp.
-	 *
-	 * @param size The size of the generated windows.
-	 * @return The time policy.
-	 */
-	public static TumblingTimeWindows of(AbstractTime size) {
-		return new TumblingTimeWindows(size.toMilliseconds());
-	}
-
-	@Override
-	public TypeSerializer<TimeWindow> getWindowSerializer(ExecutionConfig executionConfig) {
-		return new TimeWindow.Serializer();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
deleted file mode 100644
index 4b4b1ea..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.assigners;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import java.io.Serializable;
-
-import java.util.Collection;
-
-/**
- * A {@code WindowAssigner} assigns zero or more {@link Window Windows} to an element.
- *
- * <p>
- * In a window operation, elements are grouped by their key (if available) and by the windows to
- * which it was assigned. The set of elements with the same key and window is called a pane.
- * When a {@link Trigger} decides that a certain pane should fire the
- * {@link org.apache.flink.streaming.api.functions.windowing.WindowFunction} is applied
- * to produce output elements for that pane.
- *
- * @param <T> The type of elements that this WindowAssigner can assign windows to.
- * @param <W> The type of {@code Window} that this assigner assigns.
- */
-public abstract class WindowAssigner<T, W extends Window> implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Returns a {@code Collection} of windows that should be assigned to the element.
-	 *
-	 * @param element The element to which windows should be assigned.
-	 * @param timestamp The timestamp of the element.
-	 */
-	public abstract Collection<W> assignWindows(T element, long timestamp);
-
-	/**
-	 * Returns the default trigger associated with this {@code WindowAssigner}.
-	 */
-	public abstract Trigger<T, W> getDefaultTrigger(StreamExecutionEnvironment env);
-
-	/**
-	 * Returns a {@link TypeSerializer} for serializing windows that are assigned by
-	 * this {@code WindowAssigner}.
-	 */
-	public abstract TypeSerializer<W> getWindowSerializer(ExecutionConfig executionConfig);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java
deleted file mode 100644
index 0a078e2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/CountEvictor.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.evictors;
-
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * An {@link Evictor} that keeps only a certain amount of elements.
- *
- * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
- */
-public class CountEvictor<W extends Window> implements Evictor<Object, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final long maxCount;
-
-	private CountEvictor(long count) {
-		this.maxCount = count;
-	}
-
-	@Override
-	public int evict(Iterable<StreamRecord<Object>> elements, int size, W window) {
-		if (size > maxCount) {
-			return (int) (size - maxCount);
-		} else {
-			return 0;
-		}
-	}
-
-	/**
-	 * Creates a {@code CountEvictor} that keeps the given number of elements.
-	 *
-	 * @param maxCount The number of elements to keep in the pane.
-	 */
-	public static <W extends Window> CountEvictor<W> of(long maxCount) {
-		return new CountEvictor<>(maxCount);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java
deleted file mode 100644
index 0083a04..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.evictors;
-
-import com.google.common.collect.Iterables;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * An {@link Evictor} that keeps elements based on a {@link DeltaFunction} and a threshold.
- *
- * <p>
- * Eviction starts from the first element of the buffer and removes all elements from the buffer
- * which have a higher delta then the threshold. As soon as there is an element with a lower delta,
- * the eviction stops.
- *
- * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
- */
-public class DeltaEvictor<T, W extends Window> implements Evictor<T, W> {
-	private static final long serialVersionUID = 1L;
-
-	DeltaFunction<T> deltaFunction;
-	private double threshold;
-
-	private DeltaEvictor(double threshold, DeltaFunction<T> deltaFunction) {
-		this.deltaFunction = deltaFunction;
-		this.threshold = threshold;
-	}
-
-	@Override
-	public int evict(Iterable<StreamRecord<T>> elements, int size, W window) {
-		StreamRecord<T> lastElement = Iterables.getLast(elements);
-		int toEvict = 0;
-		for (StreamRecord<T> element : elements) {
-			if (deltaFunction.getDelta(element.getValue(), lastElement.getValue()) < this.threshold) {
-				break;
-			}
-			toEvict++;
-		}
-
-		return toEvict;
-	}
-
-	@Override
-	public String toString() {
-		return "DeltaEvictor(" +  deltaFunction + ", " + threshold + ")";
-	}
-
-	/**
-	 * Creates a {@code DeltaEvictor} from the given threshold and {@code DeltaFunction}.
-	 *
-	 * @param threshold The threshold
-	 * @param deltaFunction The {@code DeltaFunction}
-	 */
-	public static <T, W extends Window> DeltaEvictor<T, W> of(double threshold, DeltaFunction<T> deltaFunction) {
-		return new DeltaEvictor<>(threshold, deltaFunction);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java
deleted file mode 100644
index 5ceaf2f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/Evictor.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.evictors;
-
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import java.io.Serializable;
-
-/**
- * An {@code Evictor} can remove elements from a pane before it is being processed and after
- * window evaluation was triggered by a
- * {@link org.apache.flink.streaming.api.windowing.triggers.Trigger}.
- *
- * <p>
- * A pane is the bucket of elements that have the same key (assigned by the
- * {@link org.apache.flink.api.java.functions.KeySelector}) and same {@link Window}. An element can
- * be in multiple panes of it was assigned to multiple windows by the
- * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. These panes all
- * have their own instance of the {@code Evictor}.
- *
- * @param <T> The type of elements that this {@code Evictor} can evict.
- * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
- */
-public interface Evictor<T, W extends Window> extends Serializable {
-
-	/**
-	 * Computes how many elements should be removed from the pane. The result specifies how
-	 * many elements should be removed from the beginning.
-	 *
-	 * @param elements The elements currently in the pane.
-	 * @param size The current number of elements in the pane.
-	 * @param window The {@link Window}
-	 */
-	int evict(Iterable<StreamRecord<T>> elements, int size, W window);
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 5776d8d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.evictors;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterables;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * An {@link Evictor} that keeps elements for a certain amount of time. Elements older
- * than {@code current_time - keep_time} are evicted.
- *
- * @param <W> The type of {@link Window Windows} on which this {@code Evictor} can operate.
- */
-public class TimeEvictor<W extends Window> implements Evictor<Object, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final long windowSize;
-
-	public TimeEvictor(long windowSize) {
-		this.windowSize = windowSize;
-	}
-
-	@Override
-	public int evict(Iterable<StreamRecord<Object>> elements, int size, W window) {
-		int toEvict = 0;
-		long currentTime = Iterables.getLast(elements).getTimestamp();
-		long evictCutoff = currentTime - windowSize;
-		for (StreamRecord<Object> record: elements) {
-			if (record.getTimestamp() > evictCutoff) {
-				break;
-			}
-			toEvict++;
-		}
-		return toEvict;
-	}
-
-	@Override
-	public String toString() {
-		return "TimeEvictor(" + windowSize + ")";
-	}
-
-	@VisibleForTesting
-	public long getWindowSize() {
-		return windowSize;
-	}
-
-	/**
-	 * Creates a {@code TimeEvictor} that keeps the given number of elements.
-	 *
-	 * @param windowSize The amount of time for which to keep elements.
-	 */
-	public static <W extends Window> TimeEvictor<W> of(AbstractTime windowSize) {
-		return new TimeEvictor<>(windowSize.toMilliseconds());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 3f8fb60..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/AbstractTime.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.time;
-
-import org.apache.flink.streaming.api.TimeCharacteristic;
-
-import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Base class for {@link Time} implementations.
- */
-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/06f6ac5d/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
deleted file mode 100644
index 6a4349c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/EventTime.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.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/06f6ac5d/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
deleted file mode 100644
index 4be6ed0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/ProcessingTime.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.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/06f6ac5d/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
deleted file mode 100644
index d1b3fe3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/time/Time.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
deleted file mode 100644
index ea26309..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
+++ /dev/null
@@ -1,90 +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.triggers;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-/**
- * A {@link Trigger} that continuously fires based on a given time interval. This fires based
- * on {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks}.
- *
- * @see org.apache.flink.streaming.api.watermark.Watermark
- *
- * @param <W> The type of {@link Window Windows} on which this trigger can operate.
- */
-public class ContinuousEventTimeTrigger<W extends Window> implements Trigger<Object, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final long interval;
-
-	private ContinuousEventTimeTrigger(long interval) {
-		this.interval = interval;
-	}
-
-	@Override
-	public TriggerResult onElement(Object element, long timestamp, W window, TriggerContext ctx) throws Exception {
-
-		OperatorState<Boolean> first = ctx.getKeyValueState("first", true);
-
-		if (first.value()) {
-			long start = timestamp - (timestamp % interval);
-			long nextFireTimestamp = start + interval;
-
-			ctx.registerEventTimeTimer(nextFireTimestamp);
-
-			first.update(false);
-			return TriggerResult.CONTINUE;
-		}
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onEventTime(long time, TriggerContext ctx) {
-		ctx.registerEventTimeTimer(time + interval);
-		return TriggerResult.FIRE;
-	}
-
-	@Override
-	public TriggerResult onProcessingTime(long time,
-			TriggerContext ctx) throws Exception {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public String toString() {
-		return "ContinuousProcessingTimeTrigger(" + interval + ")";
-	}
-
-	@VisibleForTesting
-	public long getInterval() {
-		return interval;
-	}
-
-	/**
-	 * Creates a trigger that continuously fires based on the given interval.
-	 *
-	 * @param interval The time interval at which to fire.
-	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
-	 */
-	public static <W extends Window> ContinuousEventTimeTrigger<W> of(AbstractTime interval) {
-		return new ContinuousEventTimeTrigger<>(interval.toMilliseconds());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index be56738..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.triggers;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-/**
- * A {@link Trigger} that continuously fires based on a given time interval. The time is the current
- * system time.
- *
- * @param <W> The type of {@link Window Windows} on which this trigger can operate.
- */
-public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigger<Object, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final long interval;
-
-	private ContinuousProcessingTimeTrigger(long interval) {
-		this.interval = interval;
-	}
-
-	@Override
-	public TriggerResult onElement(Object element, long timestamp, W window, TriggerContext ctx) throws Exception {
-		long currentTime = System.currentTimeMillis();
-
-		OperatorState<Long> fireState = ctx.getKeyValueState("fire-timestamp", 0L);
-		long nextFireTimestamp = fireState.value();
-
-		if (nextFireTimestamp == 0) {
-			long start = currentTime - (currentTime % interval);
-			fireState.update(start + interval);
-
-			ctx.registerProcessingTimeTimer(nextFireTimestamp);
-			return TriggerResult.CONTINUE;
-		}
-		if (currentTime > nextFireTimestamp) {
-			long start = currentTime - (currentTime % interval);
-			fireState.update(start + interval);
-
-			ctx.registerProcessingTimeTimer(nextFireTimestamp);
-
-			return TriggerResult.FIRE;
-		}
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onEventTime(long time,
-			TriggerContext ctx) throws Exception {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onProcessingTime(long time, TriggerContext ctx) throws Exception {
-
-		OperatorState<Long> fireState = ctx.getKeyValueState("fire-timestamp", 0L);
-		long nextFireTimestamp = fireState.value();
-
-		// only fire if an element didn't already fire
-		long currentTime = System.currentTimeMillis();
-		if (currentTime > nextFireTimestamp) {
-			long start = currentTime - (currentTime % interval);
-			fireState.update(start + interval);
-			return TriggerResult.FIRE;
-		}
-		return TriggerResult.CONTINUE;
-	}
-
-	@VisibleForTesting
-	public long getInterval() {
-		return interval;
-	}
-
-	@Override
-	public String toString() {
-		return "ContinuousProcessingTimeTrigger(" + interval + ")";
-	}
-
-	/**
-	 * Creates a trigger that continuously fires based on the given interval.
-	 *
-	 * @param interval The time interval at which to fire.
-	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
-	 */
-	public static <W extends Window> ContinuousProcessingTimeTrigger<W> of(AbstractTime interval) {
-		return new ContinuousProcessingTimeTrigger<>(interval.toMilliseconds());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
deleted file mode 100644
index 8512989..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
+++ /dev/null
@@ -1,76 +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.triggers;
-
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-import java.io.IOException;
-
-/**
- * A {@link Trigger} that fires once the count of elements in a pane reaches the given count.
- *
- * @param <W> The type of {@link Window Windows} on which this trigger can operate.
- */
-public class CountTrigger<W extends Window> implements Trigger<Object, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final long maxCount;
-
-	private CountTrigger(long maxCount) {
-		this.maxCount = maxCount;
-	}
-
-	@Override
-	public TriggerResult onElement(Object element, long timestamp, W window, TriggerContext ctx) throws IOException {
-		OperatorState<Long> count = ctx.getKeyValueState("count", 0L);
-		long currentCount = count.value() + 1;
-		count.update(currentCount);
-		if (currentCount >= maxCount) {
-			count.update(0L);
-			return TriggerResult.FIRE;
-		}
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onEventTime(long time, TriggerContext ctx) {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onProcessingTime(long time,
-			TriggerContext ctx) throws Exception {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public String toString() {
-		return "CountTrigger(" +  maxCount + ")";
-	}
-
-	/**
-	 * Creates a trigger that fires once the number of elements in a pane reaches the given count.
-	 *
-	 * @param maxCount The count of elements at which to fire.
-	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
-	 */
-	public static <W extends Window> CountTrigger<W> of(long maxCount) {
-		return new CountTrigger<>(maxCount);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
deleted file mode 100644
index 1c6523d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
+++ /dev/null
@@ -1,89 +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.triggers;
-
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-import java.io.Serializable;
-
-/**
- * A {@link Trigger} that fires based on a {@link DeltaFunction} and a threshold.
- *
- * <p>
- * This trigger calculates a delta between the data point which triggered last
- * and the currently arrived data point. It triggers if the delta is higher than
- * a specified threshold.
- *
- * @param <W> The type of {@link Window Windows} on which this trigger can operate.
- */
-public class DeltaTrigger<T extends Serializable, W extends Window> implements Trigger<T, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final DeltaFunction<T> deltaFunction;
-	private final double threshold;
-
-	private DeltaTrigger(double threshold, DeltaFunction<T> deltaFunction) {
-		this.deltaFunction = deltaFunction;
-		this.threshold = threshold;
-	}
-
-	@Override
-	public TriggerResult onElement(T element, long timestamp, W window, TriggerContext ctx) throws Exception {
-		OperatorState<T> lastElementState = ctx.getKeyValueState("last-element", null);
-		if (lastElementState.value() == null) {
-			lastElementState.update(element);
-			return TriggerResult.CONTINUE;
-		}
-		if (deltaFunction.getDelta(lastElementState.value(), element) > this.threshold) {
-			lastElementState.update(element);
-			return TriggerResult.FIRE;
-		}
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onEventTime(long time, TriggerContext ctx) {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onProcessingTime(long time,
-			TriggerContext ctx) throws Exception {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public String toString() {
-		return "DeltaTrigger(" +  deltaFunction + ", " + threshold + ")";
-	}
-
-	/**
-	 * Creates a delta trigger from the given threshold and {@code DeltaFunction}.
-	 *
-	 * @param threshold The threshold at which to trigger.
-	 * @param deltaFunction The delta function to use
-	 *
-	 * @param <T> The type of elements on which this trigger can operate.
-	 * @param <W> The type of {@link Window Windows} on which this trigger can operate.
-	 */
-	public static <T extends Serializable, W extends Window> DeltaTrigger<T, W> of(double threshold, DeltaFunction<T> deltaFunction) {
-		return new DeltaTrigger<>(threshold, deltaFunction);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
deleted file mode 100644
index 4b6613c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.triggers;
-
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-
-/**
- * A {@link Trigger} that fires once the watermark passes the end of the window
- * to which a pane belongs.
- *
- * @see org.apache.flink.streaming.api.watermark.Watermark
- */
-public class EventTimeTrigger implements Trigger<Object, TimeWindow> {
-	private static final long serialVersionUID = 1L;
-
-	private EventTimeTrigger() {}
-
-	@Override
-	public TriggerResult onElement(Object element, long timestamp, TimeWindow window, TriggerContext ctx) throws Exception {
-		ctx.registerEventTimeTimer(window.maxTimestamp());
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onEventTime(long time, TriggerContext ctx) {
-		return TriggerResult.FIRE_AND_PURGE;
-	}
-
-	@Override
-	public TriggerResult onProcessingTime(long time,
-			TriggerContext ctx) throws Exception {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public String toString() {
-		return "EventTimeTrigger()";
-	}
-
-	/**
-	 * Creates trigger that fires once the watermark passes the end of the window.
-	 */
-	public static EventTimeTrigger create() {
-		return new EventTimeTrigger();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
deleted file mode 100644
index 6278ba6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.triggers;
-
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-
-/**
- * A {@link Trigger} that fires once the current system time passes the end of the window
- * to which a pane belongs.
- */
-public class ProcessingTimeTrigger implements Trigger<Object, TimeWindow> {
-	private static final long serialVersionUID = 1L;
-
-	private ProcessingTimeTrigger() {}
-
-	@Override
-	public TriggerResult onElement(Object element, long timestamp, TimeWindow window, TriggerContext ctx) {
-		ctx.registerProcessingTimeTimer(window.maxTimestamp());
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onEventTime(long time,
-			TriggerContext ctx) throws Exception {
-		return TriggerResult.CONTINUE;
-	}
-
-	@Override
-	public TriggerResult onProcessingTime(long time, TriggerContext ctx) {
-		return TriggerResult.FIRE_AND_PURGE;
-	}
-
-	@Override
-	public String toString() {
-		return "ProcessingTimeTrigger()";
-	}
-
-	/**
-	 * Creates a new trigger that fires once system time passes the end of the window.
-	 */
-	public static ProcessingTimeTrigger create() {
-		return new ProcessingTimeTrigger();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
deleted file mode 100644
index eaca336..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.triggers;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-/**
- * A trigger that can turn any {@link Trigger} into a purging {@code Trigger}.
- *
- * <p>
- * When the nested trigger fires, this will return a {@code FIRE_AND_PURGE}
- * {@link org.apache.flink.streaming.api.windowing.triggers.Trigger.TriggerResult}
- *
- * @param <T> The type of elements on which this trigger can operate.
- * @param <W> The type of {@link Window Windows} on which this trigger can operate.
- */
-public class PurgingTrigger<T, W extends Window> implements Trigger<T, W> {
-	private static final long serialVersionUID = 1L;
-
-	private Trigger<T, W> nestedTrigger;
-
-	private  PurgingTrigger(Trigger<T, W> nestedTrigger) {
-		this.nestedTrigger = nestedTrigger;
-	}
-
-	@Override
-	public TriggerResult onElement(T element, long timestamp, W window, TriggerContext ctx) throws Exception {
-		TriggerResult triggerResult = nestedTrigger.onElement(element, timestamp, window, ctx);
-		switch (triggerResult) {
-			case FIRE:
-				return TriggerResult.FIRE_AND_PURGE;
-			case FIRE_AND_PURGE:
-				return TriggerResult.FIRE_AND_PURGE;
-			default:
-				return TriggerResult.CONTINUE;
-		}
-	}
-
-	@Override
-	public TriggerResult onEventTime(long time, TriggerContext ctx) throws Exception {
-		TriggerResult triggerResult = nestedTrigger.onEventTime(time, ctx);
-		switch (triggerResult) {
-			case FIRE:
-				return TriggerResult.FIRE_AND_PURGE;
-			case FIRE_AND_PURGE:
-				return TriggerResult.FIRE_AND_PURGE;
-			default:
-				return TriggerResult.CONTINUE;
-		}
-	}
-
-	@Override
-	public TriggerResult onProcessingTime(long time, TriggerContext ctx) throws Exception {
-		TriggerResult triggerResult = nestedTrigger.onProcessingTime(time, ctx);
-		switch (triggerResult) {
-			case FIRE:
-				return TriggerResult.FIRE_AND_PURGE;
-			case FIRE_AND_PURGE:
-				return TriggerResult.FIRE_AND_PURGE;
-			default:
-				return TriggerResult.CONTINUE;
-		}
-	}
-
-	@Override
-	public String toString() {
-		return "PurgingTrigger(" + nestedTrigger.toString() + ")";
-	}
-
-	/**
-	 * Creates a new purging trigger from the given {@code Trigger}.
-	 *
-	 * @param nestedTrigger The trigger that is wrapped by this purging trigger
-	 */
-	public static <T, W extends Window> PurgingTrigger<T, W> of(Trigger<T, W> nestedTrigger) {
-		return new PurgingTrigger<>(nestedTrigger);
-	}
-
-	@VisibleForTesting
-	public Trigger<T, W> getNestedTrigger() {
-		return nestedTrigger;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
deleted file mode 100644
index ef8110b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.triggers;
-
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-import java.io.Serializable;
-
-/**
- * A {@code Trigger} determines when a pane of a window should be evaluated to emit the
- * results for that part of the window.
- *
- * <p>
- * A pane is the bucket of elements that have the same key (assigned by the
- * {@link org.apache.flink.api.java.functions.KeySelector}) and same {@link Window}. An element can
- * be in multiple panes of it was assigned to multiple windows by the
- * {@link org.apache.flink.streaming.api.windowing.assigners.WindowAssigner}. These panes all
- * have their own instance of the {@code Trigger}.
- *
- * <p>
- * Triggers must not maintain state internally since they can be re-created or reused for
- * different keys. All necessary state should be persisted using the state abstraction
- * available on the {@link org.apache.flink.streaming.api.windowing.triggers.Trigger.TriggerContext}.
- *
- * @param <T> The type of elements on which this {@code Trigger} works.
- * @param <W> The type of {@link Window Windows} on which this {@code Trigger} can operate.
- */
-public interface Trigger<T, W extends Window> extends Serializable {
-
-	/**
-	 * Called for every element that gets added to a pane. The result of this will determine
-	 * whether the pane is evaluated to emit results.
-	 *
-	 * @param element The element that arrived.
-	 * @param timestamp The timestamp of the element that arrived.
-	 * @param window The window to which this pane belongs.
-	 * @param ctx A context object that can be used to register timer callbacks.
-	 */
-	TriggerResult onElement(T element, long timestamp, W window, TriggerContext ctx) throws Exception;
-
-	/**
-	 * Called when a processing-time timer that was set using the trigger context fires.
-	 *
-	 * @param time The timestamp at which the timer fired.
-	 * @param ctx A context object that can be used to register timer callbacks.
-	 */
-	TriggerResult onProcessingTime(long time, TriggerContext ctx) throws Exception;
-
-	/**
-	 * Called when an event-time timer that was set using the trigger context fires.
-	 *
-	 * @param time The timestamp at which the timer fired.
-	 * @param ctx A context object that can be used to register timer callbacks.
-	 */
-	TriggerResult onEventTime(long time, TriggerContext ctx) throws Exception;
-
-
-	/**
-	 * Result type for trigger methods. This determines what happens which the window.
-	 *
-	 * <p>
-	 * On {@code FIRE} the pane is evaluated and results are emitted. The contents of the window
-	 * are kept. {@code FIRE_AND_PURGE} acts like {@code FIRE} but the contents of the pane
-	 * are purged. On {@code CONTINUE} nothing happens, processing continues.
-	 */
-	enum TriggerResult {
-		CONTINUE, FIRE_AND_PURGE, FIRE
-	}
-
-	/**
-	 * A context object that is given to {@code Trigger} methods to allow them to register timer
-	 * callbacks and deal with state.
-	 */
-	interface TriggerContext {
-
-		/**
-		 * Register a system time callback. When the current system time passes the specified
-		 * time {@link #onProcessingTime(long, TriggerContext)} is called with the time specified here.
-		 *
-		 * @param time The time at which to invoke {@link #onProcessingTime(long, TriggerContext)}
-		 */
-		void registerProcessingTimeTimer(long time);
-
-		/**
-		 * Register an event-time callback. When the current watermark passes the specified
-		 * time {@link #onEventTime(long, TriggerContext)} is called with the time specified here.
-		 *
-		 * @see org.apache.flink.streaming.api.watermark.Watermark
-		 *
-		 * @param time The watermark at which to invoke {@link #onEventTime(long, TriggerContext)}
-		 */
-		void registerEventTimeTimer(long time);
-
-		/**
-		 * Retrieves an {@link OperatorState} object that can be used to interact with
-		 * fault-tolerant state that is scoped to the window and key of the current
-		 * trigger invocation.
-		 *
-		 * @param name A unique key for the state.
-		 * @param defaultState The default value of the state.
-		 */
-		<S extends Serializable> OperatorState<S> getKeyValueState(final String name, final S defaultState);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/GlobalWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/GlobalWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/GlobalWindow.java
deleted file mode 100644
index f20c779..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/GlobalWindow.java
+++ /dev/null
@@ -1,128 +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.windows;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.IOException;
-
-public class GlobalWindow extends Window {
-
-	private static GlobalWindow INSTANCE = new GlobalWindow();
-
-	private GlobalWindow() { }
-
-	public static GlobalWindow get() {
-		return INSTANCE;
-	}
-
-	@Override
-	public long maxTimestamp() {
-		return Long.MAX_VALUE;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		return this == o || !(o == null || getClass() != o.getClass());
-	}
-
-	@Override
-	public int hashCode() {
-		return 0;
-	}
-
-	@Override
-	public String toString() {
-		return "GlobalWindow";
-	}
-
-	public static class Serializer extends TypeSerializer<GlobalWindow> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean isImmutableType() {
-			return true;
-		}
-
-		@Override
-		public TypeSerializer<GlobalWindow> duplicate() {
-			return this;
-		}
-
-		@Override
-		public GlobalWindow createInstance() {
-			return GlobalWindow.INSTANCE;
-		}
-
-		@Override
-		public GlobalWindow copy(GlobalWindow from) {
-			return from;
-		}
-
-		@Override
-		public GlobalWindow copy(GlobalWindow from, GlobalWindow reuse) {
-			return from;
-		}
-
-		@Override
-		public int getLength() {
-			return 0;
-		}
-
-		@Override
-		public void serialize(GlobalWindow record, DataOutputView target) throws IOException {
-			target.writeByte(0);
-		}
-
-		@Override
-		public GlobalWindow deserialize(DataInputView source) throws IOException {
-			source.readByte();
-			return GlobalWindow.INSTANCE;
-		}
-
-		@Override
-		public GlobalWindow deserialize(GlobalWindow reuse,
-				DataInputView source) throws IOException {
-			source.readByte();
-			return GlobalWindow.INSTANCE;
-		}
-
-		@Override
-		public void copy(DataInputView source, DataOutputView target) throws IOException {
-			source.readByte();
-			target.writeByte(0);
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			return obj instanceof Serializer;
-		}
-
-		@Override
-		public boolean canEqual(Object obj) {
-			return obj instanceof Serializer;
-		}
-
-		@Override
-		public int hashCode() {
-			return 0;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java
deleted file mode 100644
index 0c4c2a8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.windowing.windows;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.IOException;
-
-/**
- * A {@link Window} that represents a time interval from {@code start} (inclusive) to
- * {@code start + size} (exclusive).
- */
-public class TimeWindow extends Window {
-
-	private final long start;
-	private final long end;
-
-	public TimeWindow(long start, long end) {
-		this.start = start;
-		this.end = end;
-	}
-
-	public long getStart() {
-		return start;
-	}
-
-	public long getEnd() {
-		return end;
-	}
-
-	@Override
-	public long maxTimestamp() {
-		return end - 1;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		TimeWindow window = (TimeWindow) o;
-
-		return end == window.end && start == window.start;
-	}
-
-	@Override
-	public int hashCode() {
-		int result = (int) (start ^ (start >>> 32));
-		result = 31 * result + (int) (end ^ (end >>> 32));
-		return result;
-	}
-
-	@Override
-	public String toString() {
-		return "TimeWindow{" +
-				"start=" + start +
-				", end=" + end +
-				'}';
-	}
-
-	public static class Serializer extends TypeSerializer<TimeWindow> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean isImmutableType() {
-			return true;
-		}
-
-		@Override
-		public TypeSerializer<TimeWindow> duplicate() {
-			return this;
-		}
-
-		@Override
-		public TimeWindow createInstance() {
-			return null;
-		}
-
-		@Override
-		public TimeWindow copy(TimeWindow from) {
-			return from;
-		}
-
-		@Override
-		public TimeWindow copy(TimeWindow from, TimeWindow reuse) {
-			return from;
-		}
-
-		@Override
-		public int getLength() {
-			return 0;
-		}
-
-		@Override
-		public void serialize(TimeWindow record, DataOutputView target) throws IOException {
-			target.writeLong(record.start);
-			target.writeLong(record.end);
-		}
-
-		@Override
-		public TimeWindow deserialize(DataInputView source) throws IOException {
-			long start = source.readLong();
-			long end = source.readLong();
-			return new TimeWindow(start, end);
-		}
-
-		@Override
-		public TimeWindow deserialize(TimeWindow reuse, DataInputView source) throws IOException {
-			long start = source.readLong();
-			long end = source.readLong();
-			return new TimeWindow(start, end);
-		}
-
-		@Override
-		public void copy(DataInputView source, DataOutputView target) throws IOException {
-			target.writeLong(source.readLong());
-			target.writeLong(source.readLong());
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			return obj instanceof Serializer;
-		}
-
-		@Override
-		public boolean canEqual(Object obj) {
-			return obj instanceof Serializer;
-		}
-
-		@Override
-		public int hashCode() {
-			return 0;
-		}
-	}
-
-}


[23/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java
deleted file mode 100644
index 0ddf272..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
-import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests for the timer service of {@link org.apache.flink.streaming.runtime.tasks.StreamTask}.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(ResultPartitionWriter.class)
-@SuppressWarnings("serial")
-public class StreamTaskTimerTest {
-
-	@Test
-	public void testOpenCloseAndTimestamps() throws Exception {
-		final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<>();
-		
-		final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<>(mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-		StreamConfig streamConfig = testHarness.getStreamConfig();
-		
-		StreamMap<String, String> mapOperator = new StreamMap<>(new DummyMapFunction<String>());
-		streamConfig.setStreamOperator(mapOperator);
-
-		testHarness.invoke();
-
-		// first one spawns thread
-		mapTask.registerTimer(System.currentTimeMillis(), new Triggerable() {
-			@Override
-			public void trigger(long timestamp) {}
-		});
-
-		assertEquals(1, StreamTask.TRIGGER_THREAD_GROUP.activeCount());
-
-
-		testHarness.endInput();
-		testHarness.waitForTaskCompletion();
-
-		// thread needs to die in time
-		long deadline = System.currentTimeMillis() + 4000;
-		while (StreamTask.TRIGGER_THREAD_GROUP.activeCount() > 0 && System.currentTimeMillis() < deadline) {
-			Thread.sleep(10);
-		}
-
-		assertEquals("Trigger timer thread did not properly shut down",
-				0, StreamTask.TRIGGER_THREAD_GROUP.activeCount());
-	}
-	
-	@Test
-	public void checkScheduledTimestampe() {
-		try {
-			final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<>();
-			final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<>(mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-
-			StreamConfig streamConfig = testHarness.getStreamConfig();
-			StreamMap<String, String> mapOperator = new StreamMap<>(new DummyMapFunction<String>());
-			streamConfig.setStreamOperator(mapOperator);
-
-			testHarness.invoke();
-
-			final AtomicReference<Throwable> errorRef = new AtomicReference<>();
-
-			final long t1 = System.currentTimeMillis();
-			final long t2 = System.currentTimeMillis() - 200;
-			final long t3 = System.currentTimeMillis() + 100;
-			final long t4 = System.currentTimeMillis() + 200;
-
-			mapTask.registerTimer(t1, new ValidatingTriggerable(errorRef, t1, 0));
-			mapTask.registerTimer(t2, new ValidatingTriggerable(errorRef, t2, 1));
-			mapTask.registerTimer(t3, new ValidatingTriggerable(errorRef, t3, 2));
-			mapTask.registerTimer(t4, new ValidatingTriggerable(errorRef, t4, 3));
-
-			long deadline = System.currentTimeMillis() + 20000;
-			while (errorRef.get() == null &&
-					ValidatingTriggerable.numInSequence < 4 &&
-					System.currentTimeMillis() < deadline)
-			{
-				Thread.sleep(100);
-			}
-
-			// handle errors
-			if (errorRef.get() != null) {
-				errorRef.get().printStackTrace();
-				fail(errorRef.get().getMessage());
-			}
-
-			assertEquals(4, ValidatingTriggerable.numInSequence);
-
-			testHarness.endInput();
-			testHarness.waitForTaskCompletion();
-
-			// wait until the trigger thread is shut down. otherwise, the other tests may become unstable
-			deadline = System.currentTimeMillis() + 4000;
-			while (StreamTask.TRIGGER_THREAD_GROUP.activeCount() > 0 && System.currentTimeMillis() < deadline) {
-				Thread.sleep(10);
-			}
-
-			assertEquals("Trigger timer thread did not properly shut down",
-					0, StreamTask.TRIGGER_THREAD_GROUP.activeCount());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	private static class ValidatingTriggerable implements Triggerable {
-		
-		static int numInSequence;
-		
-		private final AtomicReference<Throwable> errorRef;
-		
-		private final long expectedTimestamp;
-		private final int expectedInSequence;
-
-		private ValidatingTriggerable(AtomicReference<Throwable> errorRef, long expectedTimestamp, int expectedInSequence) {
-			this.errorRef = errorRef;
-			this.expectedTimestamp = expectedTimestamp;
-			this.expectedInSequence = expectedInSequence;
-		}
-
-		@Override
-		public void trigger(long timestamp) {
-			try {
-				assertEquals(expectedTimestamp, timestamp);
-				assertEquals(expectedInSequence, numInSequence);
-				numInSequence++;
-			}
-			catch (Throwable t) {
-				errorRef.compareAndSet(null, t);
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	public static class DummyMapFunction<T> implements MapFunction<T, T> {
-		@Override
-		public T map(T value) { return value; }
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
deleted file mode 100644
index ad3c838..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ /dev/null
@@ -1,824 +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.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-import org.apache.flink.util.Collector;
-
-import org.junit.After;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.mockito.stubbing.OngoingStubbing;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import static org.mockito.Mockito.*;
-import static org.junit.Assert.*;
-
-@SuppressWarnings({"serial", "SynchronizationOnLocalVariableOrMethodParameter"})
-public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
-
-	@SuppressWarnings("unchecked")
-	private final WindowFunction<String, String, String, TimeWindow> mockFunction = mock(WindowFunction.class);
-
-	@SuppressWarnings("unchecked")
-	private final KeySelector<String, String> mockKeySelector = mock(KeySelector.class);
-	
-	private final KeySelector<Integer, Integer> identitySelector = new KeySelector<Integer, Integer>() {
-		@Override
-		public Integer getKey(Integer value) {
-			return value;
-		}
-	};
-	
-	private final WindowFunction<Integer, Integer, Integer, TimeWindow> validatingIdentityFunction =
-			new WindowFunction<Integer, Integer, Integer, TimeWindow>()
-	{
-		@Override
-		public void apply(Integer key,
-				TimeWindow window,
-				Iterable<Integer> values,
-				Collector<Integer> out) {
-			for (Integer val : values) {
-				assertEquals(key, val);
-				out.collect(val);
-			}
-		}
-	};
-
-	// ------------------------------------------------------------------------
-
-	@After
-	public void checkNoTriggerThreadsRunning() {
-		// make sure that all the threads we trigger are shut down
-		long deadline = System.currentTimeMillis() + 5000;
-		while (StreamTask.TRIGGER_THREAD_GROUP.activeCount() > 0 && System.currentTimeMillis() < deadline) {
-			try {
-				Thread.sleep(10);
-			}
-			catch (InterruptedException ignored) {}
-		}
-
-		assertTrue("Not all trigger threads where properly shut down",
-				StreamTask.TRIGGER_THREAD_GROUP.activeCount() == 0);
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void testInvalidParameters() {
-		try {
-			assertInvalidParameter(-1L, -1L);
-			assertInvalidParameter(10000L, -1L);
-			assertInvalidParameter(-1L, 1000L);
-			assertInvalidParameter(1000L, 2000L);
-			
-			// actual internal slide is too low here:
-			assertInvalidParameter(1000L, 999L);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testWindowSizeAndSlide() {
-		try {
-			AccumulatingProcessingTimeWindowOperator<String, String, String> op;
-			
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-			assertEquals(5000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(5, op.getNumPanesPerWindow());
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-			assertEquals(1000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(1, op.getNumPanesPerWindow());
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-			assertEquals(1500, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(500, op.getPaneSize());
-			assertEquals(3, op.getNumPanesPerWindow());
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-			assertEquals(1200, op.getWindowSize());
-			assertEquals(1100, op.getWindowSlide());
-			assertEquals(100, op.getPaneSize());
-			assertEquals(12, op.getNumPanesPerWindow());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testWindowTriggerTimeAlignment() {
-		try {
-			@SuppressWarnings("unchecked")
-			final Output<StreamRecord<String>> mockOut = mock(Output.class);
-			final StreamTask<?, ?> mockTask = createMockTask();
-
-			AccumulatingProcessingTimeWindowOperator<String, String, String> op;
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			op.dispose();
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			op.dispose();
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 500 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			op.dispose();
-
-			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 100 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1100 == 0);
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testTumblingWindow() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final int windowSize = 50;
-			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			final int numElements = 1000;
-
-			for (int i = 0; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> result = out.getElements();
-			assertEquals(numElements, result.size());
-
-			Collections.sort(result);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, result.get(i).intValue());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void testSlidingWindow() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-			
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			final int numElements = 1000;
-
-			for (int i = 0; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> result = out.getElements();
-
-			// if we kept this running, each element would be in the result three times (for each slide).
-			// we are closing the window before the final panes are through three times, so we may have less
-			// elements.
-			if (result.size() < numElements || result.size() > 3 * numElements) {
-				fail("Wrong number of results: " + result.size());
-			}
-
-			Collections.sort(result);
-			int lastNum = -1;
-			int lastCount = -1;
-			
-			for (int num : result) {
-				if (num == lastNum) {
-					lastCount++;
-					assertTrue(lastCount <= 3);
-				}
-				else {
-					lastNum = num;
-					lastCount = 1;
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void testTumblingWindowSingleElements() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE, 50, 50);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			synchronized (lock) {
-				op.processElement(new StreamRecord<Integer>(1));
-				op.processElement(new StreamRecord<Integer>(2));
-			}
-			out.waitForNElements(2, 60000);
-
-			synchronized (lock) {
-				op.processElement(new StreamRecord<Integer>(3));
-				op.processElement(new StreamRecord<Integer>(4));
-				op.processElement(new StreamRecord<Integer>(5));
-			}
-			out.waitForNElements(5, 60000);
-
-			synchronized (lock) {
-				op.processElement(new StreamRecord<Integer>(6));
-			}
-			out.waitForNElements(6, 60000);
-			
-			List<Integer> result = out.getElements();
-			assertEquals(6, result.size());
-
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), result);
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-	
-	@Test
-	public void testSlidingWindowSingleElements() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// tumbling window that triggers every 20 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			synchronized (lock) {
-				op.processElement(new StreamRecord<Integer>(1));
-				op.processElement(new StreamRecord<Integer>(2));
-			}
-
-			// each element should end up in the output three times
-			// wait until the elements have arrived 6 times in the output
-			out.waitForNElements(6, 120000);
-			
-			List<Integer> result = out.getElements();
-			assertEquals(6, result.size());
-			
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 1, 1, 2, 2, 2), result);
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-	
-	@Test
-	public void testEmitTrailingDataOnClose() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>();
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-			
-			// the operator has a window time that is so long that it will not fire in this test
-			final long oneYear = 365L * 24 * 60 * 60 * 1000;
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							oneYear, oneYear);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-			
-			List<Integer> data = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
-			for (Integer i : data) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-			
-			// get and verify the result
-			List<Integer> result = out.getElements();
-			Collections.sort(result);
-			assertEquals(data, result);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void testPropagateExceptionsFromClose() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>();
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			WindowFunction<Integer, Integer, Integer, TimeWindow> failingFunction = new FailingFunction(100);
-
-			// the operator has a window time that is so long that it will not fire in this test
-			final long hundredYears = 100L * 365 * 24 * 60 * 60 * 1000;
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							failingFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							hundredYears, hundredYears);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			for (int i = 0; i < 150; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-			}
-			
-			try {
-				synchronized (lock) {
-					op.close();
-				}
-				fail("This should fail with an exception");
-			}
-			catch (Exception e) {
-				assertTrue(
-						e.getMessage().contains("Artificial Test Exception") ||
-						(e.getCause() != null && e.getCause().getMessage().contains("Artificial Test Exception")));
-			}
-
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-	
-	@Test
-	public void checkpointRestoreWithPendingWindowTumbling() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final int windowSize = 200;
-			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// tumbling window that triggers every 50 milliseconds
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			// inject some elements
-			final int numElementsFirst = 700;
-			for (int i = 0; i < numElementsFirst; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			// draw a snapshot and dispose the window
-			StreamTaskState state;
-			List<Integer> resultAtSnapshot;
-			synchronized (lock) {
-				int beforeSnapShot = out.getElements().size(); 
-				state = op.snapshotOperatorState(1L, System.currentTimeMillis());
-				resultAtSnapshot = new ArrayList<>(out.getElements());
-				int afterSnapShot = out.getElements().size();
-				assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-			}
-
-			// inject some random elements, which should not show up in the state
-			for (int i = 0; i < 300; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i + numElementsFirst));
-				}
-				Thread.sleep(1);
-			}
-			
-			op.dispose();
-			
-			// re-create the operator and restore the state
-			final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSize);
-			op = new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out2);
-			op.restoreState(state);
-			op.open();
-
-			// inject some more elements
-			final int numElements = 1000;
-			for (int i = numElementsFirst; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
-			finalResult.addAll(out2.getElements());
-			assertEquals(numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, finalResult.get(i).intValue());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowSliding() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final int factor = 4;
-			final int windowSlide = 50;
-			final int windowSize = factor * windowSlide;
-			
-			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSlide);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// sliding window (200 msecs) every 50 msecs
-			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
-					new AccumulatingProcessingTimeWindowOperator<>(
-							validatingIdentityFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSlide);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			// inject some elements
-			final int numElements = 1000;
-			final int numElementsFirst = 700;
-			
-			for (int i = 0; i < numElementsFirst; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			// draw a snapshot
-			StreamTaskState state;
-			List<Integer> resultAtSnapshot;
-			synchronized (lock) {
-				int beforeSnapShot = out.getElements().size();
-				state = op.snapshotOperatorState(1L, System.currentTimeMillis());
-				resultAtSnapshot = new ArrayList<>(out.getElements());
-				int afterSnapShot = out.getElements().size();
-				assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-			}
-			
-			assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);
-
-			// inject the remaining elements - these should not influence the snapshot
-			for (int i = numElementsFirst; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-			
-			op.dispose();
-			
-			// re-create the operator and restore the state
-			final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSlide);
-			op = new AccumulatingProcessingTimeWindowOperator<>(
-					validatingIdentityFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-					windowSize, windowSlide);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out2);
-			op.restoreState(state);
-			op.open();
-			
-
-			// inject again the remaining elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			// for a deterministic result, we need to wait until all pending triggers
-			// have fired and emitted their results
-			long deadline = System.currentTimeMillis() + 120000;
-			do {
-				Thread.sleep(20);
-			}
-			while (resultAtSnapshot.size() + out2.getElements().size() < factor * numElements
-					&& System.currentTimeMillis() < deadline);
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
-			finalResult.addAll(out2.getElements());
-			assertEquals(factor * numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < factor * numElements; i++) {
-				assertEquals(i / factor, finalResult.get(i).intValue());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private void assertInvalidParameter(long windowSize, long windowSlide) {
-		try {
-			new AccumulatingProcessingTimeWindowOperator<String, String, String>(
-					mockFunction, mockKeySelector, 
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE,
-					windowSize, windowSlide);
-			fail("This should fail with an IllegalArgumentException");
-		}
-		catch (IllegalArgumentException e) {
-			// expected
-		}
-		catch (Exception e) {
-			fail("Wrong exception. Expected IllegalArgumentException but found " + e.getClass().getSimpleName());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	private static class FailingFunction implements WindowFunction<Integer, Integer, Integer, TimeWindow> {
-
-		private final int failAfterElements;
-		
-		private int numElements;
-
-		FailingFunction(int failAfterElements) {
-			this.failAfterElements = failAfterElements;
-		}
-
-		@Override
-		public void apply(Integer integer,
-				TimeWindow window,
-				Iterable<Integer> values,
-				Collector<Integer> out) throws Exception {
-			for (Integer i : values) {
-				out.collect(i);
-				numElements++;
-				
-				if (numElements >= failAfterElements) {
-					throw new Exception("Artificial Test Exception");
-				}
-			}
-		}
-	}
-
-	private static StreamTask<?, ?> createMockTask() {
-		StreamTask<?, ?> task = mock(StreamTask.class);
-		when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
-		when(task.getName()).thenReturn("Test task name");
-		when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
-
-		Environment env = mock(Environment.class);
-		when(env.getIndexInSubtaskGroup()).thenReturn(0);
-		when(env.getNumberOfSubtasks()).thenReturn(1);
-		when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader());
-
-		when(task.getEnvironment()).thenReturn(env);
-
-		// ugly java generic hacks to get the state backend into the mock
-		@SuppressWarnings("unchecked")
-		OngoingStubbing<StateBackend<?>> stubbing =
-				(OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(task.getStateBackend());
-		stubbing.thenReturn(MemoryStateBackend.defaultInstance());
-		
-		return task;
-	}
-
-	private static StreamTask<?, ?> createMockTaskWithTimer(
-			final ScheduledExecutorService timerService, final Object lock)
-	{
-		StreamTask<?, ?> mockTask = createMockTask();
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
-				final Long timestamp = (Long) invocationOnMock.getArguments()[0];
-				final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
-				timerService.schedule(
-						new Callable<Object>() {
-							@Override
-							public Object call() throws Exception {
-								synchronized (lock) {
-									target.trigger(timestamp);
-								}
-								return null;
-							}
-						},
-						timestamp - System.currentTimeMillis(),
-						TimeUnit.MILLISECONDS);
-				return null;
-			}
-		}).when(mockTask).registerTimer(anyLong(), any(Triggerable.class));
-
-		return mockTask;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
deleted file mode 100644
index 4bd260f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ /dev/null
@@ -1,823 +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.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.StringSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-import org.junit.After;
-import org.junit.Test;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.mockito.stubbing.OngoingStubbing;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-@SuppressWarnings({"serial", "SynchronizationOnLocalVariableOrMethodParameter"})
-public class AggregatingAlignedProcessingTimeWindowOperatorTest {
-
-	@SuppressWarnings("unchecked")
-	private final ReduceFunction<String> mockFunction = mock(ReduceFunction.class);
-
-	@SuppressWarnings("unchecked")
-	private final KeySelector<String, String> mockKeySelector = mock(KeySelector.class);
-	
-	private final KeySelector<Integer, Integer> identitySelector = new KeySelector<Integer, Integer>() {
-		@Override
-		public Integer getKey(Integer value) {
-			return value;
-		}
-	};
-	
-	private final ReduceFunction<Integer> sumFunction = new ReduceFunction<Integer>() {
-		@Override
-		public Integer reduce(Integer value1, Integer value2) {
-			return value1 + value2;
-		}
-	};
-
-	// ------------------------------------------------------------------------
-
-	@After
-	public void checkNoTriggerThreadsRunning() {
-		// make sure that all the threads we trigger are shut down
-		long deadline = System.currentTimeMillis() + 5000;
-		while (StreamTask.TRIGGER_THREAD_GROUP.activeCount() > 0 && System.currentTimeMillis() < deadline) {
-			try {
-				Thread.sleep(10);
-			}
-			catch (InterruptedException ignored) {}
-		}
-
-		assertTrue("Not all trigger threads where properly shut down",
-				StreamTask.TRIGGER_THREAD_GROUP.activeCount() == 0);
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void testInvalidParameters() {
-		try {
-			assertInvalidParameter(-1L, -1L);
-			assertInvalidParameter(10000L, -1L);
-			assertInvalidParameter(-1L, 1000L);
-			assertInvalidParameter(1000L, 2000L);
-			
-			// actual internal slide is too low here:
-			assertInvalidParameter(1000L, 999L);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testWindowSizeAndSlide() {
-		try {
-			AggregatingProcessingTimeWindowOperator<String, String> op;
-			
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-			assertEquals(5000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(5, op.getNumPanesPerWindow());
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-			assertEquals(1000, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(1000, op.getPaneSize());
-			assertEquals(1, op.getNumPanesPerWindow());
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-			assertEquals(1500, op.getWindowSize());
-			assertEquals(1000, op.getWindowSlide());
-			assertEquals(500, op.getPaneSize());
-			assertEquals(3, op.getNumPanesPerWindow());
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-			assertEquals(1200, op.getWindowSize());
-			assertEquals(1100, op.getWindowSlide());
-			assertEquals(100, op.getPaneSize());
-			assertEquals(12, op.getNumPanesPerWindow());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testWindowTriggerTimeAlignment() {
-		try {
-			@SuppressWarnings("unchecked")
-			final Output<StreamRecord<String>> mockOut = mock(Output.class);
-			final StreamTask<?, ?> mockTask = createMockTask();
-			
-			AggregatingProcessingTimeWindowOperator<String, String> op;
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			op.dispose();
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 1000 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			op.dispose();
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 500 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
-			op.dispose();
-
-			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
-			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
-			op.open();
-			assertTrue(op.getNextSlideTime() % 100 == 0);
-			assertTrue(op.getNextEvaluationTime() % 1100 == 0);
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testTumblingWindowUniqueElements() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final int windowSize = 50;
-			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-			
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-			
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-			
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			final int numElements = 1000;
-
-			for (int i = 0; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> result = out.getElements();
-			assertEquals(numElements, result.size());
-
-			Collections.sort(result);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, result.get(i).intValue());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdownNow();
-		}
-	}
-
-	@Test
-	public void  testTumblingWindowDuplicateElements() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-
-		try {
-			final int windowSize = 50;
-			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-			
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-			
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			final int numWindows = 10;
-
-			long previousNextTime = 0;
-			int window = 1;
-			
-			while (window <= numWindows) {
-				synchronized (lock) {
-					long nextTime = op.getNextEvaluationTime();
-					int val = ((int) nextTime) ^ ((int) (nextTime >>> 32));
-					
-					op.processElement(new StreamRecord<Integer>(val));
-
-					if (nextTime != previousNextTime) {
-						window++;
-						previousNextTime = nextTime;
-					}
-				}
-				Thread.sleep(1);
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-			
-			List<Integer> result = out.getElements();
-			
-			// we have ideally one element per window. we may have more, when we emitted a value into the
-			// successive window (corner case), so we can have twice the number of elements, in the worst case.
-			assertTrue(result.size() >= numWindows && result.size() <= 2 * numWindows);
-
-			// deduplicate for more accurate checks
-			HashSet<Integer> set = new HashSet<>(result);
-			assertTrue(set.size() == 10);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void testSlidingWindow() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// tumbling window that triggers every 20 milliseconds
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							150, 50);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			final int numElements = 1000;
-
-			for (int i = 0; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> result = out.getElements();
-			
-			// every element can occur between one and three times
-			if (result.size() < numElements || result.size() > 3 * numElements) {
-				System.out.println(result);
-				fail("Wrong number of results: " + result.size());
-			}
-
-			Collections.sort(result);
-			int lastNum = -1;
-			int lastCount = -1;
-			
-			for (int num : result) {
-				if (num == lastNum) {
-					lastCount++;
-					assertTrue(lastCount <= 3);
-				}
-				else {
-					lastNum = num;
-					lastCount = 1;
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdownNow();
-		}
-	}
-
-	@Test
-	public void testSlidingWindowSingleElements() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// tumbling window that triggers every 20 milliseconds
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			synchronized (lock) {
-				op.processElement(new StreamRecord<Integer>(1));
-				op.processElement(new StreamRecord<Integer>(2));
-			}
-
-			// each element should end up in the output three times
-			// wait until the elements have arrived 6 times in the output
-			out.waitForNElements(6, 120000);
-			
-			List<Integer> result = out.getElements();
-			assertEquals(6, result.size());
-			
-			Collections.sort(result);
-			assertEquals(Arrays.asList(1, 1, 1, 2, 2, 2), result);
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-	
-	@Test
-	public void testEmitTrailingDataOnClose() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>();
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-			
-			// the operator has a window time that is so long that it will not fire in this test
-			final long oneYear = 365L * 24 * 60 * 60 * 1000;
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op = 
-					new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE, oneYear, oneYear);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-			
-			List<Integer> data = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
-			for (Integer i : data) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-			
-			// get and verify the result
-			List<Integer> result = out.getElements();
-			Collections.sort(result);
-			assertEquals(data, result);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void testPropagateExceptionsFromProcessElement() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final CollectingOutput<Integer> out = new CollectingOutput<>();
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			ReduceFunction<Integer> failingFunction = new FailingFunction(100);
-
-			// the operator has a window time that is so long that it will not fire in this test
-			final long hundredYears = 100L * 365 * 24 * 60 * 60 * 1000;
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							failingFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							hundredYears, hundredYears);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			for (int i = 0; i < 100; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(1));
-				}
-			}
-			
-			try {
-				op.processElement(new StreamRecord<Integer>(1));
-				fail("This fail with an exception");
-			}
-			catch (Exception e) {
-				assertTrue(e.getMessage().contains("Artificial Test Exception"));
-			}
-
-			op.dispose();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowTumbling() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final int windowSize = 200;
-			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// tumbling window that triggers every 50 milliseconds
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSize);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			// inject some elements
-			final int numElementsFirst = 700;
-			final int numElements = 1000;
-			
-			for (int i = 0; i < numElementsFirst; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			// draw a snapshot and dispose the window
-			StreamTaskState state;
-			List<Integer> resultAtSnapshot;
-			synchronized (lock) {
-				int beforeSnapShot = out.getElements().size();
-				state = op.snapshotOperatorState(1L, System.currentTimeMillis());
-				resultAtSnapshot = new ArrayList<>(out.getElements());
-				int afterSnapShot = out.getElements().size();
-				assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-			}
-			
-			assertTrue(resultAtSnapshot.size() <= numElementsFirst);
-
-			// inject some random elements, which should not show up in the state
-			for (int i = numElementsFirst; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			op.dispose();
-
-			// re-create the operator and restore the state
-			final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSize);
-			op = new AggregatingProcessingTimeWindowOperator<>(
-					sumFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-					windowSize, windowSize);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out2);
-			op.restoreState(state);
-			op.open();
-
-			// inject the remaining elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
-			finalResult.addAll(out2.getElements());
-			assertEquals(numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(i, finalResult.get(i).intValue());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-
-	@Test
-	public void checkpointRestoreWithPendingWindowSliding() {
-		final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
-		try {
-			final int factor = 4;
-			final int windowSlide = 50;
-			final int windowSize = factor * windowSlide;
-
-			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSlide);
-			final Object lock = new Object();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-
-			// sliding window (200 msecs) every 50 msecs
-			AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
-					new AggregatingProcessingTimeWindowOperator<>(
-							sumFunction, identitySelector,
-							IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-							windowSize, windowSlide);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out);
-			op.open();
-
-			// inject some elements
-			final int numElements = 1000;
-			final int numElementsFirst = 700;
-
-			for (int i = 0; i < numElementsFirst; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			// draw a snapshot
-			StreamTaskState state;
-			List<Integer> resultAtSnapshot;
-			synchronized (lock) {
-				int beforeSnapShot = out.getElements().size();
-				state = op.snapshotOperatorState(1L, System.currentTimeMillis());
-				resultAtSnapshot = new ArrayList<>(out.getElements());
-				int afterSnapShot = out.getElements().size();
-				assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
-			}
-
-			assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);
-
-			// inject the remaining elements - these should not influence the snapshot
-			for (int i = numElementsFirst; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			op.dispose();
-
-			// re-create the operator and restore the state
-			final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSlide);
-			op = new AggregatingProcessingTimeWindowOperator<>(
-					sumFunction, identitySelector,
-					IntSerializer.INSTANCE, IntSerializer.INSTANCE,
-					windowSize, windowSlide);
-
-			op.setup(mockTask, new StreamConfig(new Configuration()), out2);
-			op.restoreState(state);
-			op.open();
-
-
-			// inject again the remaining elements
-			for (int i = numElementsFirst; i < numElements; i++) {
-				synchronized (lock) {
-					op.processElement(new StreamRecord<Integer>(i));
-				}
-				Thread.sleep(1);
-			}
-
-			// for a deterministic result, we need to wait until all pending triggers
-			// have fired and emitted their results
-			long deadline = System.currentTimeMillis() + 120000;
-			do {
-				Thread.sleep(20);
-			}
-			while (resultAtSnapshot.size() + out2.getElements().size() < factor * numElements
-					&& System.currentTimeMillis() < deadline);
-
-			synchronized (lock) {
-				op.close();
-			}
-			op.dispose();
-
-			// get and verify the result
-			List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
-			finalResult.addAll(out2.getElements());
-			assertEquals(factor * numElements, finalResult.size());
-
-			Collections.sort(finalResult);
-			for (int i = 0; i < factor * numElements; i++) {
-				assertEquals(i / factor, finalResult.get(i).intValue());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			timerService.shutdown();
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private void assertInvalidParameter(long windowSize, long windowSlide) {
-		try {
-			new AggregatingProcessingTimeWindowOperator<String, String>(
-					mockFunction, mockKeySelector,
-					StringSerializer.INSTANCE, StringSerializer.INSTANCE,
-					windowSize, windowSlide);
-			fail("This should fail with an IllegalArgumentException");
-		}
-		catch (IllegalArgumentException e) {
-			// expected
-		}
-		catch (Exception e) {
-			fail("Wrong exception. Expected IllegalArgumentException but found " + e.getClass().getSimpleName());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	private static class FailingFunction implements ReduceFunction<Integer> {
-
-		private final int failAfterElements;
-		
-		private int numElements;
-
-		FailingFunction(int failAfterElements) {
-			this.failAfterElements = failAfterElements;
-		}
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			numElements++;
-
-			if (numElements >= failAfterElements) {
-				throw new Exception("Artificial Test Exception");
-			}
-			
-			return value1 + value2;
-		}
-	}
-	
-	private static StreamTask<?, ?> createMockTask() {
-		StreamTask<?, ?> task = mock(StreamTask.class);
-		when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
-		when(task.getName()).thenReturn("Test task name");
-		when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
-
-		Environment env = mock(Environment.class);
-		when(env.getIndexInSubtaskGroup()).thenReturn(0);
-		when(env.getNumberOfSubtasks()).thenReturn(1);
-		when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader());
-		
-		when(task.getEnvironment()).thenReturn(env);
-
-		// ugly java generic hacks to get the state backend into the mock
-		@SuppressWarnings("unchecked")
-		OngoingStubbing<StateBackend<?>> stubbing =
-				(OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(task.getStateBackend());
-		stubbing.thenReturn(MemoryStateBackend.defaultInstance());
-		
-		return task;
-	}
-
-	private static StreamTask<?, ?> createMockTaskWithTimer(
-			final ScheduledExecutorService timerService, final Object lock)
-	{
-		StreamTask<?, ?> mockTask = createMockTask();
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
-				final Long timestamp = (Long) invocationOnMock.getArguments()[0];
-				final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
-				timerService.schedule(
-						new Callable<Object>() {
-							@Override
-							public Object call() throws Exception {
-								synchronized (lock) {
-									target.trigger(timestamp);
-								}
-								return null;
-							}
-						},
-						timestamp - System.currentTimeMillis(),
-						TimeUnit.MILLISECONDS);
-				return null;
-			}
-		}).when(mockTask).registerTimer(anyLong(), any(Triggerable.class));
-		
-		return mockTask;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
deleted file mode 100644
index 282c71f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
+++ /dev/null
@@ -1,215 +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.Tuple2;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-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.evictors.CountEvictor;
-import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger;
-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;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * These tests verify that the api calls on
- * {@link org.apache.flink.streaming.api.datastream.AllWindowedStream} instantiate
- * the correct window operator.
- */
-public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
-
-	/**
-	 * These tests ensure that the correct trigger is set when using event-time windows.
-	 */
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testEventTime() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
-		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
-
-		DummyReducer reducer = new DummyReducer();
-
-		DataStream<Tuple2<String, Integer>> window1 = source
-				.windowAll(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.reduce(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.assertFalse(winOperator1.isSetProcessingTime());
-		Assert.assertTrue(winOperator1.getTrigger() instanceof EventTimeTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(
-							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.assertFalse(winOperator2.isSetProcessingTime());
-		Assert.assertTrue(winOperator2.getTrigger() instanceof EventTimeTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testNonEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		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(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.trigger(CountTrigger.of(100))
-				.reduce(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.isSetProcessingTime());
-		Assert.assertTrue(winOperator1.getTrigger() instanceof CountTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-				.trigger(CountTrigger.of(100))
-				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(
-							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(winOperator1.isSetProcessingTime());
-		Assert.assertTrue(winOperator2.getTrigger() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
-		Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-	}
-
-	@Test
-	@SuppressWarnings("rawtypes")
-	public void testEvicting() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
-
-		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(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
-				.evictor(CountEvictor.of(100))
-				.reduce(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.assertFalse(winOperator1.isSetProcessingTime());
-		Assert.assertTrue(winOperator1.getTrigger() instanceof EventTimeTrigger);
-		Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
-		Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
-		Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
-
-		DataStream<Tuple2<String, Integer>> window2 = source
-				.windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-				.trigger(CountTrigger.of(100))
-				.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
-				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(
-							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.assertFalse(winOperator2.isSetProcessingTime());
-		Assert.assertTrue(winOperator2.getTrigger() instanceof CountTrigger);
-		Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
-		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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
deleted file mode 100644
index cfae026..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
+++ /dev/null
@@ -1,373 +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 com.google.common.collect.Lists;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
-
-	private static List<String> testResults;
-
-	@Test
-	public void testCoGroup() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple2.of("a", 0));
-				ctx.collect(Tuple2.of("a", 1));
-				ctx.collect(Tuple2.of("a", 2));
-
-				ctx.collect(Tuple2.of("b", 3));
-				ctx.collect(Tuple2.of("b", 4));
-				ctx.collect(Tuple2.of("b", 5));
-
-				ctx.collect(Tuple2.of("a", 6));
-				ctx.collect(Tuple2.of("a", 7));
-				ctx.collect(Tuple2.of("a", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).assignTimestamps(new Tuple2TimestampExtractor());
-
-		DataStream<Tuple2<String, Integer>> source2 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple2.of("a", 0));
-				ctx.collect(Tuple2.of("a", 1));
-
-				ctx.collect(Tuple2.of("b", 3));
-
-				ctx.collect(Tuple2.of("c", 6));
-				ctx.collect(Tuple2.of("c", 7));
-				ctx.collect(Tuple2.of("c", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).assignTimestamps(new Tuple2TimestampExtractor());
-
-
-		source1.coGroup(source2)
-				.where(new Tuple2KeyExtractor())
-				.equalTo(new Tuple2KeyExtractor())
-				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
-					@Override
-					public void coGroup(Iterable<Tuple2<String, Integer>> first,
-							Iterable<Tuple2<String, Integer>> second,
-							Collector<String> out) throws Exception {
-						StringBuilder result = new StringBuilder();
-						result.append("F:");
-						for (Tuple2<String, Integer> t: first) {
-							result.append(t.toString());
-						}
-						result.append(" S:");
-						for (Tuple2<String, Integer> t: second) {
-							result.append(t.toString());
-						}
-						out.collect(result.toString());
-					}
-				})
-				.addSink(new SinkFunction<String>() {
-					@Override
-					public void invoke(String value) throws Exception {
-						testResults.add(value);
-					}
-				});
-
-		env.execute("CoGroup Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
-				"F:(b,3)(b,4)(b,5) S:(b,3)",
-				"F:(a,6)(a,7)(a,8) S:",
-				"F: S:(c,6)(c,7)(c,8)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	@Test
-	public void testJoin() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple3.of("a", "x", 0));
-				ctx.collect(Tuple3.of("a", "y", 1));
-				ctx.collect(Tuple3.of("a", "z", 2));
-
-				ctx.collect(Tuple3.of("b", "u", 3));
-				ctx.collect(Tuple3.of("b", "w", 5));
-
-				ctx.collect(Tuple3.of("a", "i", 6));
-				ctx.collect(Tuple3.of("a", "j", 7));
-				ctx.collect(Tuple3.of("a", "k", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).assignTimestamps(new Tuple3TimestampExtractor());
-
-		DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple3.of("a", "u", 0));
-				ctx.collect(Tuple3.of("a", "w", 1));
-
-				ctx.collect(Tuple3.of("b", "i", 3));
-				ctx.collect(Tuple3.of("b", "k", 5));
-
-				ctx.collect(Tuple3.of("a", "x", 6));
-				ctx.collect(Tuple3.of("a", "z", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).assignTimestamps(new Tuple3TimestampExtractor());
-
-
-		source1.join(source2)
-				.where(new Tuple3KeyExtractor())
-				.equalTo(new Tuple3KeyExtractor())
-				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
-					@Override
-					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
-						return first + ":" + second;
-					}
-				})
-				.addSink(new SinkFunction<String>() {
-					@Override
-					public void invoke(String value) throws Exception {
-						testResults.add(value);
-					}
-				});
-
-		env.execute("Join Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"(a,x,0):(a,u,0)",
-				"(a,x,0):(a,w,1)",
-				"(a,y,1):(a,u,0)",
-				"(a,y,1):(a,w,1)",
-				"(a,z,2):(a,u,0)",
-				"(a,z,2):(a,w,1)",
-				"(b,u,3):(b,i,3)",
-				"(b,u,3):(b,k,5)",
-				"(b,w,5):(b,i,3)",
-				"(b,w,5):(b,k,5)",
-				"(a,i,6):(a,x,6)",
-				"(a,i,6):(a,z,8)",
-				"(a,j,7):(a,x,6)",
-				"(a,j,7):(a,z,8)",
-				"(a,k,8):(a,x,6)",
-				"(a,k,8):(a,z,8)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	@Test
-	public void testSelfJoin() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple3.of("a", "x", 0));
-				ctx.collect(Tuple3.of("a", "y", 1));
-				ctx.collect(Tuple3.of("a", "z", 2));
-
-				ctx.collect(Tuple3.of("b", "u", 3));
-				ctx.collect(Tuple3.of("b", "w", 5));
-
-				ctx.collect(Tuple3.of("a", "i", 6));
-				ctx.collect(Tuple3.of("a", "j", 7));
-				ctx.collect(Tuple3.of("a", "k", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).assignTimestamps(new Tuple3TimestampExtractor());
-
-		source1.join(source1)
-				.where(new Tuple3KeyExtractor())
-				.equalTo(new Tuple3KeyExtractor())
-				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
-					@Override
-					public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
-						return first + ":" + second;
-					}
-				})
-				.addSink(new SinkFunction<String>() {
-					@Override
-					public void invoke(String value) throws Exception {
-						testResults.add(value);
-					}
-				});
-
-		env.execute("Self-Join Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"(a,x,0):(a,x,0)",
-				"(a,x,0):(a,y,1)",
-				"(a,x,0):(a,z,2)",
-				"(a,y,1):(a,x,0)",
-				"(a,y,1):(a,y,1)",
-				"(a,y,1):(a,z,2)",
-				"(a,z,2):(a,x,0)",
-				"(a,z,2):(a,y,1)",
-				"(a,z,2):(a,z,2)",
-				"(b,u,3):(b,u,3)",
-				"(b,u,3):(b,w,5)",
-				"(b,w,5):(b,u,3)",
-				"(b,w,5):(b,w,5)",
-				"(a,i,6):(a,i,6)",
-				"(a,i,6):(a,j,7)",
-				"(a,i,6):(a,k,8)",
-				"(a,j,7):(a,i,6)",
-				"(a,j,7):(a,j,7)",
-				"(a,j,7):(a,k,8)",
-				"(a,k,8):(a,i,6)",
-				"(a,k,8):(a,j,7)",
-				"(a,k,8):(a,k,8)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
-			return element.f1;
-		}
-
-		@Override
-		public long extractWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
-			return element.f1 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-
-	private static class Tuple3TimestampExtractor implements TimestampExtractor<Tuple3<String, String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple3<String, String, Integer> element, long currentTimestamp) {
-			return element.f2;
-		}
-
-		@Override
-		public long extractWatermark(Tuple3<String, String, Integer> element, long currentTimestamp) {
-			return element.f2 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-
-	private static class Tuple2KeyExtractor 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;
-		}
-	}
-
-	private static class Tuple3KeyExtractor implements KeySelector<Tuple3<String, String, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple3<String, String, Integer> value) throws Exception {
-			return value.f0;
-		}
-	}
-
-}


[26/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
deleted file mode 100644
index 8038cfb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import static org.junit.Assert.*;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
-import org.junit.Test;
-
-@SuppressWarnings("serial")
-public class SlotAllocationTest {
-	
-	@Test
-	public void test() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		FilterFunction<Long> dummyFilter = new FilterFunction<Long>() {
-			@Override
-			public boolean filter(Long value) { return false; }
-		};
-
-		env.generateSequence(1, 10).filter(dummyFilter).isolateResources().filter(dummyFilter)
-				.disableChaining().filter(dummyFilter).startNewResourceGroup().filter(dummyFilter)
-				.startNewChain().print();
-
-		JobGraph jobGraph = env.getStreamGraph().getJobGraph();
-
-		List<JobVertex> vertices = jobGraph.getVerticesSortedTopologicallyFromSources();
-
-		assertEquals(vertices.get(0).getSlotSharingGroup(), vertices.get(2).getSlotSharingGroup());
-		assertNotEquals(vertices.get(0).getSlotSharingGroup(), vertices.get(1).getSlotSharingGroup());
-		assertNotEquals(vertices.get(2).getSlotSharingGroup(), vertices.get(3).getSlotSharingGroup());
-		assertEquals(vertices.get(3).getSlotSharingGroup(), vertices.get(4).getSlotSharingGroup());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
deleted file mode 100644
index c316604..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.graph;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.datastream.ConnectedStreams;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.util.EvenOddOutputSelector;
-import org.apache.flink.streaming.util.NoOpIntMap;
-import org.apache.flink.streaming.util.NoOpSink;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for {@link StreamGraphGenerator}. This only tests correct translation of split/select,
- * union, partitioning since the other translation routines are tested already in operation
- * specific tests, for example in {@link org.apache.flink.streaming.api.IterateTest} for
- * iterations.
- */
-public class StreamGraphGeneratorTest extends StreamingMultipleProgramsTestBase {
-
-	/**
-	 * This tests whether virtual Transformations behave correctly.
-	 *
-	 * <p>
-	 * Verifies that partitioning, output selector, selected names are correctly set in the
-	 * StreamGraph when they are intermixed.
-	 */
-	@Test
-	public void testVirtualTransformations() throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source = env.fromElements(1, 10);
-
-		DataStream<Integer> rebalanceMap = source.rebalance().map(new NoOpIntMap());
-
-		// verify that only the partitioning that was set last is used
-		DataStream<Integer> broadcastMap = rebalanceMap
-				.forward()
-				.global()
-				.broadcast()
-				.map(new NoOpIntMap());
-
-		broadcastMap.addSink(new NoOpSink<Integer>());
-
-		// verify that partitioning is preserved across union and split/select
-		EvenOddOutputSelector selector1 = new EvenOddOutputSelector();
-		EvenOddOutputSelector selector2 = new EvenOddOutputSelector();
-		EvenOddOutputSelector selector3 = new EvenOddOutputSelector();
-
-		DataStream<Integer> map1Operator = rebalanceMap
-				.map(new NoOpIntMap());
-
-		DataStream<Integer> map1 = map1Operator
-				.broadcast()
-				.split(selector1)
-				.select("even");
-
-		DataStream<Integer> map2Operator = rebalanceMap
-				.map(new NoOpIntMap());
-
-		DataStream<Integer> map2 = map2Operator
-				.split(selector2)
-				.select("odd")
-				.global();
-
-		DataStream<Integer> map3Operator = rebalanceMap
-				.map(new NoOpIntMap());
-
-		DataStream<Integer> map3 = map3Operator
-				.global()
-				.split(selector3)
-				.select("even")
-				.shuffle();
-
-
-		SingleOutputStreamOperator<Integer, ?> unionedMap = map1.union(map2).union(map3)
-				.map(new NoOpIntMap());
-
-		unionedMap.addSink(new NoOpSink<Integer>());
-
-		StreamGraph graph = env.getStreamGraph();
-
-		// rebalanceMap
-		assertTrue(graph.getStreamNode(rebalanceMap.getId()).getInEdges().get(0).getPartitioner() instanceof RebalancePartitioner);
-
-		// verify that only last partitioning takes precedence
-		assertTrue(graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
-		assertEquals(rebalanceMap.getId(), graph.getStreamNode(broadcastMap.getId()).getInEdges().get(0).getSourceVertex().getId());
-
-		// verify that partitioning in unions is preserved and that it works across split/select
-		assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
-		assertTrue(graph.getStreamNode(map1Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even"));
-		assertTrue(graph.getStreamNode(map1Operator.getId()).getOutputSelectors().contains(selector1));
-
-		assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof GlobalPartitioner);
-		assertTrue(graph.getStreamNode(map2Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("odd"));
-		assertTrue(graph.getStreamNode(map2Operator.getId()).getOutputSelectors().contains(selector2));
-
-		assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getPartitioner() instanceof ShufflePartitioner);
-		assertTrue(graph.getStreamNode(map3Operator.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("even"));
-		assertTrue(graph.getStreamNode(map3Operator.getId()).getOutputSelectors().contains(selector3));
-	}
-
-	/**
-	 * This tests whether virtual Transformations behave correctly.
-	 *
-	 * Checks whether output selector, partitioning works correctly when applied on a union.
-	 */
-	@Test
-	public void testVirtualTransformations2() throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source = env.fromElements(1, 10);
-
-		DataStream<Integer> rebalanceMap = source.rebalance().map(new NoOpIntMap());
-
-		DataStream<Integer> map1 = rebalanceMap
-				.map(new NoOpIntMap());
-
-		DataStream<Integer> map2 = rebalanceMap
-				.map(new NoOpIntMap());
-
-		DataStream<Integer> map3 = rebalanceMap
-				.map(new NoOpIntMap());
-
-		EvenOddOutputSelector selector = new EvenOddOutputSelector();
-
-		SingleOutputStreamOperator<Integer, ?> unionedMap = map1.union(map2).union(map3)
-				.broadcast()
-				.split(selector)
-				.select("foo")
-				.map(new NoOpIntMap());
-
-		unionedMap.addSink(new NoOpSink<Integer>());
-
-		StreamGraph graph = env.getStreamGraph();
-
-		// verify that the properties are correctly set on all input operators
-		assertTrue(graph.getStreamNode(map1.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
-		assertTrue(graph.getStreamNode(map1.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo"));
-		assertTrue(graph.getStreamNode(map1.getId()).getOutputSelectors().contains(selector));
-
-		assertTrue(graph.getStreamNode(map2.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
-		assertTrue(graph.getStreamNode(map2.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo"));
-		assertTrue(graph.getStreamNode(map2.getId()).getOutputSelectors().contains(selector));
-
-		assertTrue(graph.getStreamNode(map3.getId()).getOutEdges().get(0).getPartitioner() instanceof BroadcastPartitioner);
-		assertTrue(graph.getStreamNode(map3.getId()).getOutEdges().get(0).getSelectedNames().get(0).equals("foo"));
-		assertTrue(graph.getStreamNode(map3.getId()).getOutputSelectors().contains(selector));
-
-	}
-
-	/**
-	 * Test whether an {@link OutputTypeConfigurable} implementation gets called with the correct
-	 * output type. In this test case the output type must be BasicTypeInfo.INT_TYPE_INFO.
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void testOutputTypeConfigurationWithOneInputTransformation() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source = env.fromElements(1, 10);
-
-		OutputTypeConfigurableOperationWithOneInput outputTypeConfigurableOperation = new OutputTypeConfigurableOperationWithOneInput();
-
-		DataStream<Integer> result = source.transform(
-			"Single input and output type configurable operation",
-			BasicTypeInfo.INT_TYPE_INFO,
-			outputTypeConfigurableOperation);
-
-		result.addSink(new NoOpSink<Integer>());
-
-		StreamGraph graph = env.getStreamGraph();
-
-		assertEquals(BasicTypeInfo.INT_TYPE_INFO, outputTypeConfigurableOperation.getTypeInformation());
-	}
-
-	@Test
-	public void testOutputTypeConfigurationWithTwoInputTransformation() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source1 = env.fromElements(1, 10);
-		DataStream<Integer> source2 = env.fromElements(2, 11);
-
-		ConnectedStreams<Integer, Integer> connectedSource = source1.connect(source2);
-
-		OutputTypeConfigurableOperationWithTwoInputs outputTypeConfigurableOperation = new OutputTypeConfigurableOperationWithTwoInputs();
-
-		DataStream<Integer> result = connectedSource.transform(
-				"Two input and output type configurable operation",
-				BasicTypeInfo.INT_TYPE_INFO,
-				outputTypeConfigurableOperation);
-
-		result.addSink(new NoOpSink<Integer>());
-
-		StreamGraph graph = env.getStreamGraph();
-
-		assertEquals(BasicTypeInfo.INT_TYPE_INFO, outputTypeConfigurableOperation.getTypeInformation());
-	}
-
-	private static class OutputTypeConfigurableOperationWithTwoInputs
-			extends AbstractStreamOperator<Integer>
-			implements TwoInputStreamOperator<Integer, Integer, Integer>, OutputTypeConfigurable<Integer> {
-
-		TypeInformation<Integer> tpeInformation;
-
-		public TypeInformation<Integer> getTypeInformation() {
-			return tpeInformation;
-		}
-
-		@Override
-		public void setOutputType(TypeInformation<Integer> outTypeInfo, ExecutionConfig executionConfig) {
-			tpeInformation = outTypeInfo;
-		}
-
-		@Override
-		public void processElement1(StreamRecord element) throws Exception {
-			output.collect(element);
-		}
-
-		@Override
-		public void processElement2(StreamRecord element) throws Exception {
-			output.collect(element);
-		}
-
-		@Override
-		public void processWatermark1(Watermark mark) throws Exception {}
-
-		@Override
-		public void processWatermark2(Watermark mark) throws Exception {}
-
-		@Override
-		public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<Integer>> output) {}
-	}
-
-	private static class OutputTypeConfigurableOperationWithOneInput
-			extends AbstractStreamOperator<Integer>
-			implements OneInputStreamOperator<Integer, Integer>, OutputTypeConfigurable<Integer> {
-
-		TypeInformation<Integer> tpeInformation;
-
-		public TypeInformation<Integer> getTypeInformation() {
-			return tpeInformation;
-		}
-
-		@Override
-		public void processElement(StreamRecord<Integer> element) throws Exception {
-			output.collect(element);
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) throws Exception {
-
-		}
-
-		@Override
-		public void setOutputType(TypeInformation<Integer> outTypeInfo, ExecutionConfig executionConfig) {
-			tpeInformation = outTypeInfo;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
deleted file mode 100644
index e806428..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.graph;
-
-import java.io.IOException;
-import java.util.Random;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StreamingJobGraphGeneratorTest extends StreamingMultipleProgramsTestBase {
-	private static final Logger LOG = LoggerFactory.getLogger(StreamingJobGraphGeneratorTest.class);
-	
-	@Test
-	public void testExecutionConfigSerialization() throws IOException, ClassNotFoundException {
-		final long seed = System.currentTimeMillis();
-		LOG.info("Test seed: {}", new Long(seed));
-		final Random r = new Random(seed);
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		StreamGraph streamingJob = new StreamGraph(env);
-		StreamingJobGraphGenerator compiler = new StreamingJobGraphGenerator(streamingJob);
-		
-		boolean closureCleanerEnabled = r.nextBoolean(), forceAvroEnabled = r.nextBoolean(), forceKryoEnabled = r.nextBoolean(), objectReuseEnabled = r.nextBoolean(), sysoutLoggingEnabled = r.nextBoolean();
-		int dop = 1 + r.nextInt(10);
-		
-		ExecutionConfig config = streamingJob.getExecutionConfig();
-		if(closureCleanerEnabled) {
-			config.enableClosureCleaner();
-		} else {
-			config.disableClosureCleaner();
-		}
-		if(forceAvroEnabled) {
-			config.enableForceAvro();
-		} else {
-			config.disableForceAvro();
-		}
-		if(forceKryoEnabled) {
-			config.enableForceKryo();
-		} else {
-			config.disableForceKryo();
-		}
-		if(objectReuseEnabled) {
-			config.enableObjectReuse();
-		} else {
-			config.disableObjectReuse();
-		}
-		if(sysoutLoggingEnabled) {
-			config.enableSysoutLogging();
-		} else {
-			config.disableSysoutLogging();
-		}
-		config.setParallelism(dop);
-		
-		JobGraph jobGraph = compiler.createJobGraph("test");
-		ExecutionConfig executionConfig = (ExecutionConfig) InstantiationUtil.readObjectFromConfig(
-				jobGraph.getJobConfiguration(),
-				ExecutionConfig.CONFIG_KEY,
-				Thread.currentThread().getContextClassLoader());
-		
-		Assert.assertEquals(closureCleanerEnabled, executionConfig.isClosureCleanerEnabled());
-		Assert.assertEquals(forceAvroEnabled, executionConfig.isForceAvroEnabled());
-		Assert.assertEquals(forceKryoEnabled, executionConfig.isForceKryoEnabled());
-		Assert.assertEquals(objectReuseEnabled, executionConfig.isObjectReuseEnabled());
-		Assert.assertEquals(sysoutLoggingEnabled, executionConfig.isSysoutLoggingEnabled());
-		Assert.assertEquals(dop, executionConfig.getParallelism());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java
deleted file mode 100644
index dc8024c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamCounterTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.streaming.api.watermark.Watermark;
-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.Test;
-
-/**
- * Tests for {@link StreamCounter}. These test that:
- *
- * <ul>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-public class StreamCounterTest {
-
-	@Test
-	public void testCount() throws Exception {
-		StreamCounter<String> operator = new StreamCounter<String>();
-
-		OneInputStreamOperatorTestHarness<String, Long> testHarness = new OneInputStreamOperatorTestHarness<String, Long>(operator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<String>("eins", initialTime + 1));
-		testHarness.processElement(new StreamRecord<String>("zwei", initialTime + 2));
-		testHarness.processWatermark(new Watermark(initialTime + 2));
-		testHarness.processElement(new StreamRecord<String>("drei", initialTime + 3));
-
-		expectedOutput.add(new StreamRecord<Long>(1L, initialTime + 1));
-		expectedOutput.add(new StreamRecord<Long>(2L, initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<Long>(3L, initialTime + 3));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
deleted file mode 100644
index 047aad8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.watermark.Watermark;
-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;
-
-/**
- * Tests for {@link StreamFilter}. These test that:
- *
- * <ul>
- *     <li>RichFunction methods are called correctly</li>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-public class StreamFilterTest {
-
-	static class MyFilter implements FilterFunction<Integer> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Integer value) throws Exception {
-			return value % 2 == 0;
-		}
-	}
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testFilter() throws Exception {
-		StreamFilter<Integer> operator = new StreamFilter<Integer>(new MyFilter());
-
-		OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<Integer, Integer>(operator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<Integer>(1, initialTime + 1));
-		testHarness.processElement(new StreamRecord<Integer>(2, initialTime + 2));
-		testHarness.processWatermark(new Watermark(initialTime + 2));
-		testHarness.processElement(new StreamRecord<Integer>(3, initialTime + 3));
-		testHarness.processElement(new StreamRecord<Integer>(4, initialTime + 4));
-		testHarness.processElement(new StreamRecord<Integer>(5, initialTime + 5));
-		testHarness.processElement(new StreamRecord<Integer>(6, initialTime + 6));
-		testHarness.processElement(new StreamRecord<Integer>(7, initialTime + 7));
-
-		expectedOutput.add(new StreamRecord<Integer>(2, initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<Integer>(4, initialTime + 4));
-		expectedOutput.add(new StreamRecord<Integer>(6, initialTime + 6));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	@Test
-	public void testOpenClose() throws Exception {
-		StreamFilter<String> operator = new StreamFilter<String>(new TestOpenCloseFilterFunction());
-
-		OneInputStreamOperatorTestHarness<String, String> testHarness = new OneInputStreamOperatorTestHarness<String, String>(operator);
-
-		long initialTime = 0L;
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<String>("fooHello", initialTime));
-		testHarness.processElement(new StreamRecord<String>("bar", initialTime));
-
-		testHarness.close();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseFilterFunction.closeCalled);
-		Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseFilterFunction extends RichFilterFunction<String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public boolean filter(String value) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return value.startsWith("foo");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java
deleted file mode 100644
index e4e29c1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFlatMapTest.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for {@link StreamMap}. These test that:
- *
- * <ul>
- *     <li>RichFunction methods are called correctly</li>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-public class StreamFlatMapTest {
-
-	public static final class MyFlatMap implements FlatMapFunction<Integer, Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(Integer value, Collector<Integer> out) throws Exception {
-			if (value % 2 == 0) {
-				out.collect(value);
-				out.collect(value * value);
-			}
-		}
-	}
-
-	@Test
-	public void testFlatMap() throws Exception {
-		StreamFlatMap<Integer, Integer> operator = new StreamFlatMap<Integer, Integer>(new MyFlatMap());
-
-		OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<Integer, Integer>(operator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<Integer>(1, initialTime + 1));
-		testHarness.processElement(new StreamRecord<Integer>(2, initialTime + 2));
-		testHarness.processWatermark(new Watermark(initialTime + 2));
-		testHarness.processElement(new StreamRecord<Integer>(3, initialTime + 3));
-		testHarness.processElement(new StreamRecord<Integer>(4, initialTime + 4));
-		testHarness.processElement(new StreamRecord<Integer>(5, initialTime + 5));
-		testHarness.processElement(new StreamRecord<Integer>(6, initialTime + 6));
-		testHarness.processElement(new StreamRecord<Integer>(7, initialTime + 7));
-		testHarness.processElement(new StreamRecord<Integer>(8, initialTime + 8));
-
-		expectedOutput.add(new StreamRecord<Integer>(2, initialTime + 2));
-		expectedOutput.add(new StreamRecord<Integer>(4, initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<Integer>(4, initialTime + 4));
-		expectedOutput.add(new StreamRecord<Integer>(16, initialTime + 4));
-		expectedOutput.add(new StreamRecord<Integer>(6, initialTime + 6));
-		expectedOutput.add(new StreamRecord<Integer>(36, initialTime + 6));
-		expectedOutput.add(new StreamRecord<Integer>(8, initialTime + 8));
-		expectedOutput.add(new StreamRecord<Integer>(64, initialTime + 8));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	@Test
-	public void testOpenClose() throws Exception {
-		StreamFlatMap<String, String> operator = new StreamFlatMap<String, String>(new TestOpenCloseFlatMapFunction());
-
-		OneInputStreamOperatorTestHarness<String, String> testHarness = new OneInputStreamOperatorTestHarness<String, String>(operator);
-
-		long initialTime = 0L;
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<String>("Hello", initialTime));
-
-		testHarness.close();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseFlatMapFunction.closeCalled);
-		Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseFlatMapFunction extends RichFlatMapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public void flatMap(String value, Collector<String> out) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			out.collect(value);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
deleted file mode 100644
index f6e7e6b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.RichFoldFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.watermark.Watermark;
-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.Test;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for {@link StreamGroupedFold}. These test that:
- *
- * <ul>
- *     <li>RichFunction methods are called correctly</li>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-@SuppressWarnings("serial")
-public class StreamGroupedFoldTest {
-
-	private static class MyFolder implements FoldFunction<Integer, String> {
-
-		@Override
-		public String fold(String accumulator, Integer value) throws Exception {
-			return accumulator + value.toString();
-		}
-	}
-
-	@Test
-	public void testGroupedFold() throws Exception {
-
-		KeySelector<Integer, String> keySelector = new KeySelector<Integer, String>() {
-			
-			@Override
-			public String getKey(Integer value) {
-				return value.toString();
-			}
-		};
-		
-		StreamGroupedFold<Integer, String, String> operator = new StreamGroupedFold<>(new MyFolder(), "100");
-		operator.setOutputType(BasicTypeInfo.STRING_TYPE_INFO, new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Integer, String> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
-		testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.STRING_TYPE_INFO);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
-		testHarness.processElement(new StreamRecord<>(1, initialTime + 2));
-		testHarness.processWatermark(new Watermark(initialTime + 2));
-		testHarness.processElement(new StreamRecord<>(2, initialTime + 3));
-		testHarness.processElement(new StreamRecord<>(2, initialTime + 4));
-		testHarness.processElement(new StreamRecord<>(3, initialTime + 5));
-
-		expectedOutput.add(new StreamRecord<>("1001", initialTime + 1));
-		expectedOutput.add(new StreamRecord<>("10011", initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<>("1002", initialTime + 3));
-		expectedOutput.add(new StreamRecord<>("10022", initialTime + 4));
-		expectedOutput.add(new StreamRecord<>("1003", initialTime + 5));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	@Test
-	public void testOpenClose() throws Exception {
-		KeySelector<Integer, Integer> keySelector = new KeySelector<Integer, Integer>() {
-			@Override
-			public Integer getKey(Integer value) {
-				return value;
-			}
-		};
-		
-		StreamGroupedFold<Integer, String, Integer> operator = new StreamGroupedFold<>(
-				new TestOpenCloseFoldFunction(), "init");
-		operator.setOutputType(BasicTypeInfo.STRING_TYPE_INFO, new ExecutionConfig());
-
-		OneInputStreamOperatorTestHarness<Integer, String> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
-		testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);
-		
-		
-		long initialTime = 0L;
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<>(1, initialTime));
-		testHarness.processElement(new StreamRecord<>(2, initialTime));
-
-		testHarness.close();
-
-		assertTrue("RichFunction methods where not called.", TestOpenCloseFoldFunction.closeCalled);
-		assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseFoldFunction extends RichFoldFunction<Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public String fold(String acc, Integer in) throws Exception {
-			if (!openCalled) {
-				fail("Open was not called before run.");
-			}
-			return acc + in;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
deleted file mode 100644
index 6cb46c9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.watermark.Watermark;
-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;
-
-/**
- * Tests for {@link StreamGroupedReduce}. These test that:
- *
- * <ul>
- *     <li>RichFunction methods are called correctly</li>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-
-public class StreamGroupedReduceTest {
-
-	@Test
-	public void testGroupedReduce() throws Exception {
-
-		KeySelector<Integer, Integer> keySelector = new IntegerKeySelector();
-		
-		StreamGroupedReduce<Integer> operator = new StreamGroupedReduce<>(new MyReducer(), IntSerializer.INSTANCE);
-
-		OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
-		testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<>(1, initialTime + 1));
-		testHarness.processElement(new StreamRecord<>(1, initialTime + 2));
-		testHarness.processWatermark(new Watermark(initialTime + 2));
-		testHarness.processElement(new StreamRecord<>(2, initialTime + 3));
-		testHarness.processElement(new StreamRecord<>(2, initialTime + 4));
-		testHarness.processElement(new StreamRecord<>(3, initialTime + 5));
-
-		expectedOutput.add(new StreamRecord<>(1, initialTime + 1));
-		expectedOutput.add(new StreamRecord<>(2, initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<>(2, initialTime + 3));
-		expectedOutput.add(new StreamRecord<>(4, initialTime + 4));
-		expectedOutput.add(new StreamRecord<>(3, initialTime + 5));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	@Test
-	public void testOpenClose() throws Exception {
-
-		KeySelector<Integer, Integer> keySelector = new IntegerKeySelector();
-		
-		StreamGroupedReduce<Integer> operator =
-				new StreamGroupedReduce<>(new TestOpenCloseReduceFunction(), IntSerializer.INSTANCE);
-		OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
-		testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);
-
-		long initialTime = 0L;
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<>(1, initialTime));
-		testHarness.processElement(new StreamRecord<>(2, initialTime));
-
-		testHarness.close();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseReduceFunction.closeCalled);
-		Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseReduceFunction extends RichReduceFunction<Integer> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public Integer reduce(Integer in1, Integer in2) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return in1 + in2;
-		}
-	}
-
-	// Utilities
-
-	private static class MyReducer implements ReduceFunction<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer reduce(Integer value1, Integer value2) throws Exception {
-			return value1 + value2;
-		}
-
-	}
-
-	private static class IntegerKeySelector implements KeySelector<Integer, Integer> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Integer getKey(Integer value) throws Exception {
-			return value;
-		}
-	}
-
-	private static TypeInformation<Integer> typeInfo = BasicTypeInfo.INT_TYPE_INFO;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
deleted file mode 100644
index f0113d1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.watermark.Watermark;
-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;
-
-/**
- * Tests for {@link StreamMap}. These test that:
- *
- * <ul>
- *     <li>RichFunction methods are called correctly</li>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-public class StreamMapTest {
-
-	private static class Map implements MapFunction<Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map(Integer value) throws Exception {
-			return "+" + (value + 1);
-		}
-	}
-	
-	@Test
-	public void testMap() throws Exception {
-		StreamMap<Integer, String> operator = new StreamMap<Integer, String>(new Map());
-
-		OneInputStreamOperatorTestHarness<Integer, String> testHarness = new OneInputStreamOperatorTestHarness<Integer, String>(operator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<Integer>(1, initialTime + 1));
-		testHarness.processElement(new StreamRecord<Integer>(2, initialTime + 2));
-		testHarness.processWatermark(new Watermark(initialTime + 2));
-		testHarness.processElement(new StreamRecord<Integer>(3, initialTime + 3));
-
-		expectedOutput.add(new StreamRecord<String>("+2", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("+3", initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("+4", initialTime + 3));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	@Test
-	public void testOpenClose() throws Exception {
-		StreamMap<String, String> operator = new StreamMap<String, String>(new TestOpenCloseMapFunction());
-
-		OneInputStreamOperatorTestHarness<String, String> testHarness = new OneInputStreamOperatorTestHarness<String, String>(operator);
-
-		long initialTime = 0L;
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<String>("Hello", initialTime));
-
-		testHarness.close();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseMapFunction.closeCalled);
-		Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseMapFunction extends RichMapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public String map(String value) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return value;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
deleted file mode 100644
index 14abd18..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.util.HashSet;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
-import org.apache.flink.streaming.api.datastream.StreamProjection;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-
-import org.junit.Test;
-
-/**
- * Tests for {@link StreamProject}. These test that:
- *
- * <ul>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-public class StreamProjectTest extends StreamingMultipleProgramsTestBase {
-
-	@Test
-	public void testProject() throws Exception {
-
-		TypeInformation<Tuple5<Integer, String, Integer, String, Integer>> inType = TypeExtractor
-				.getForObject(new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "b", 4));
-
-		int[] fields = new int[]{4, 4, 3};
-
-		TupleSerializer<Tuple3<Integer, Integer, String>> serializer =
-				new TupleTypeInfo<Tuple3<Integer, Integer, String>>(StreamProjection.extractFieldTypes(fields, inType))
-						.createSerializer(new ExecutionConfig());
-		@SuppressWarnings("unchecked")
-		StreamProject<Tuple5<Integer, String, Integer, String, Integer>, Tuple3<Integer, Integer, String>> operator =
-				new StreamProject<Tuple5<Integer, String, Integer, String, Integer>, Tuple3<Integer, Integer, String>>(
-						fields, serializer);
-
-		OneInputStreamOperatorTestHarness<Tuple5<Integer, String, Integer, String, Integer>, Tuple3<Integer, Integer, String>> testHarness = new OneInputStreamOperatorTestHarness<Tuple5<Integer, String, Integer, String, Integer>, Tuple3<Integer, Integer, String>>(operator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.open();
-
-		testHarness.processElement(new StreamRecord<Tuple5<Integer, String, Integer, String, Integer>>(new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "b", 4), initialTime + 1));
-		testHarness.processElement(new StreamRecord<Tuple5<Integer, String, Integer, String, Integer>>(new Tuple5<Integer, String, Integer, String, Integer>(2, "s", 3, "c", 2), initialTime + 2));
-		testHarness.processElement(new StreamRecord<Tuple5<Integer, String, Integer, String, Integer>>(new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "c", 2), initialTime + 3));
-		testHarness.processWatermark(new Watermark(initialTime + 2));
-		testHarness.processElement(new StreamRecord<Tuple5<Integer, String, Integer, String, Integer>>(new Tuple5<Integer, String, Integer, String, Integer>(2, "a", 3, "a", 7), initialTime + 4));
-
-		expectedOutput.add(new StreamRecord<Tuple3<Integer, Integer, String>>(new Tuple3<Integer, Integer, String>(4, 4, "b"), initialTime + 1));
-		expectedOutput.add(new StreamRecord<Tuple3<Integer, Integer, String>>(new Tuple3<Integer, Integer, String>(2, 2, "c"), initialTime + 2));
-		expectedOutput.add(new StreamRecord<Tuple3<Integer, Integer, String>>(new Tuple3<Integer, Integer, String>(2, 2, "c"), initialTime + 3));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<Tuple3<Integer, Integer, String>>(new Tuple3<Integer, Integer, String>(7, 7, "a"), initialTime + 4));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-
-	// tests using projection from the API without explicitly specifying the types
-	private static HashSet<Tuple2<Long, Double>> expected = new HashSet<Tuple2<Long, Double>>();
-	private static HashSet<Tuple2<Long, Double>> actual = new HashSet<Tuple2<Long, Double>>();
-
-	@Test
-	public void APIWithoutTypesTest() {
-
-		for (Long i = 1L; i < 11L; i++) {
-			expected.add(new Tuple2<Long, Double>(i, i.doubleValue()));
-		}
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		env.generateSequence(1, 10).map(new MapFunction<Long, Tuple3<Long, Character, Double>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Tuple3<Long, Character, Double> map(Long value) throws Exception {
-				return new Tuple3<Long, Character, Double>(value, 'c', value.doubleValue());
-			}
-		})
-			.project(0, 2)
-			.addSink(new SinkFunction<Tuple>() {
-				private static final long serialVersionUID = 1L;
-
-				@Override
-				@SuppressWarnings("unchecked")
-				public void invoke(Tuple value) throws Exception {
-					actual.add( (Tuple2<Long,Double>) value);
-				}
-			});
-
-		try {
-			env.execute();
-		} catch (Exception e) {
-			fail(e.getMessage());
-		}
-
-		assertEquals(expected, actual);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java
deleted file mode 100644
index 39e85e9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoGroupedReduceTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-///*
-// * Licensed to the Apache Software Foundation (ASF) under one or more
-// * contributor license agreements.  See the NOTICE file distributed with
-// * this work for additional information regarding copyright ownership.
-// * The ASF licenses this file to You under the Apache License, Version 2.0
-// * (the "License"); you may not use this file except in compliance with
-// * the License.  You may obtain a copy of the License at
-// *
-// *    http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.flink.streaming.api.operators.co;
-//
-//import static org.junit.Assert.assertEquals;
-//
-//import java.util.Arrays;
-//import java.util.List;
-//
-//import org.apache.flink.api.java.functions.KeySelector;
-//import org.apache.flink.api.java.tuple.Tuple2;
-//import org.apache.flink.api.java.tuple.Tuple3;
-//import org.apache.flink.streaming.api.functions.co.CoReduceFunction;
-//import org.apache.flink.streaming.api.operators.co.CoStreamGroupedReduce;
-//import org.apache.flink.streaming.util.MockCoContext;
-//import org.junit.Test;
-//
-//public class CoGroupedReduceTest {
-//
-//	private final static class MyCoReduceFunction implements
-//			CoReduceFunction<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String> {
-//		private static final long serialVersionUID = 1L;
-//
-//		@Override
-//		public Tuple3<String, String, String> reduce1(Tuple3<String, String, String> value1,
-//				Tuple3<String, String, String> value2) {
-//			return new Tuple3<String, String, String>(value1.f0, value1.f1 + value2.f1, value1.f2);
-//		}
-//
-//		@Override
-//		public Tuple2<Integer, Integer> reduce2(Tuple2<Integer, Integer> value1,
-//				Tuple2<Integer, Integer> value2) {
-//			return new Tuple2<Integer, Integer>(value1.f0, value1.f1 + value2.f1);
-//		}
-//
-//		@Override
-//		public String map1(Tuple3<String, String, String> value) {
-//			return value.f1;
-//		}
-//
-//		@Override
-//		public String map2(Tuple2<Integer, Integer> value) {
-//			return value.f1.toString();
-//		}
-//	}
-//
-//	@SuppressWarnings("unchecked")
-//	@Test
-//	public void coGroupedReduceTest() {
-//		Tuple3<String, String, String> word1 = new Tuple3<String, String, String>("a", "word1", "b");
-//		Tuple3<String, String, String> word2 = new Tuple3<String, String, String>("b", "word2", "a");
-//		Tuple3<String, String, String> word3 = new Tuple3<String, String, String>("a", "word3", "a");
-//		Tuple2<Integer, Integer> int1 = new Tuple2<Integer, Integer>(2, 1);
-//		Tuple2<Integer, Integer> int2 = new Tuple2<Integer, Integer>(1, 2);
-//		Tuple2<Integer, Integer> int3 = new Tuple2<Integer, Integer>(0, 3);
-//		Tuple2<Integer, Integer> int4 = new Tuple2<Integer, Integer>(2, 4);
-//		Tuple2<Integer, Integer> int5 = new Tuple2<Integer, Integer>(1, 5);
-//
-//		KeySelector<Tuple3<String, String, String>, ?> keySelector0 = new KeySelector<Tuple3<String, String, String>, String>() {
-//
-//			private static final long serialVersionUID = 1L;
-//
-//			@Override
-//			public String getKey(Tuple3<String, String, String> value) throws Exception {
-//				return value.f0;
-//			}
-//		};
-//
-//		KeySelector<Tuple2<Integer, Integer>, ?> keySelector1 = new KeySelector<Tuple2<Integer, Integer>, Integer>() {
-//
-//			private static final long serialVersionUID = 1L;
-//
-//			@Override
-//			public Integer getKey(Tuple2<Integer, Integer> value) throws Exception {
-//				return value.f0;
-//			}
-//		};
-//
-//		KeySelector<Tuple3<String, String, String>, ?> keySelector2 = new KeySelector<Tuple3<String, String, String>, String>() {
-//
-//			private static final long serialVersionUID = 1L;
-//
-//			@Override
-//			public String getKey(Tuple3<String, String, String> value) throws Exception {
-//				return value.f2;
-//			}
-//		};
-//
-//		CoStreamGroupedReduce<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String> invokable = new CoStreamGroupedReduce<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String>(
-//				new MyCoReduceFunction(), keySelector0, keySelector1);
-//
-//		List<String> expected = Arrays.asList("word1", "1", "word2", "2", "word1word3", "3", "5",
-//				"7");
-//
-//		List<String> actualList = MockCoContext.createAndExecute(invokable,
-//				Arrays.asList(word1, word2, word3), Arrays.asList(int1, int2, int3, int4, int5));
-//
-//		assertEquals(expected, actualList);
-//
-//		invokable = new CoStreamGroupedReduce<Tuple3<String, String, String>, Tuple2<Integer, Integer>, String>(
-//				new MyCoReduceFunction(), keySelector2, keySelector1);
-//
-//		expected = Arrays.asList("word1", "1", "word2", "2", "word2word3", "3", "5", "7");
-//
-//		actualList = MockCoContext.createAndExecute(invokable, Arrays.asList(word1, word2, word3),
-//				Arrays.asList(int1, int2, int3, int4, int5));
-//
-//		assertEquals(expected, actualList);
-//	}
-//}


[24/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
deleted file mode 100644
index d4fdc59..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
+++ /dev/null
@@ -1,954 +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.io;
-
-import org.apache.flink.core.memory.HeapMemorySegment;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.util.Arrays;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for the behavior of the {@link BarrierBuffer}.
- */
-public class BarrierBufferTest {
-
-	private static final int PAGE_SIZE = 512;
-	
-	private static int SIZE_COUNTER = 0;
-	
-	private static IOManager IO_MANAGER;
-
-	@BeforeClass
-	public static void setup() {
-		IO_MANAGER = new IOManagerAsync();
-		SIZE_COUNTER = 1;
-	}
-
-	@AfterClass
-	public static void shutdownIOManager() {
-		IO_MANAGER.shutdown();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Tests
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Validates that the buffer behaves correctly if no checkpoint barriers come,
-	 * for a single input channel.
-	 */
-	@Test
-	public void testSingleChannelNoBarriers() {
-		try {
-			BufferOrEvent[] sequence = { 
-					createBuffer(0), createBuffer(0), createBuffer(0),
-					createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			for (BufferOrEvent boe : sequence) {
-				assertEquals(boe, buffer.getNextNonBlocked());
-			}
-			
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-			
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Validates that the buffer behaves correctly if no checkpoint barriers come,
-	 * for an input with multiple input channels.
-	 */
-	@Test
-	public void testMultiChannelNoBarriers() {
-		try {
-			BufferOrEvent[] sequence = { createBuffer(2), createBuffer(2), createBuffer(0),
-					createBuffer(1), createBuffer(0), createEndOfPartition(0),
-					createBuffer(3), createBuffer(1), createEndOfPartition(3),
-					createBuffer(1), createEndOfPartition(1), createBuffer(2), createEndOfPartition(2)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 4, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			for (BufferOrEvent boe : sequence) {
-				assertEquals(boe, buffer.getNextNonBlocked());
-			}
-
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Validates that the buffer preserved the order of elements for a 
-	 * input with a single input channel, and checkpoint events.
-	 */
-	@Test
-	public void testSingleChannelWithBarriers() {
-		try {
-			BufferOrEvent[] sequence = {
-					createBuffer(0), createBuffer(0), createBuffer(0),
-					createBarrier(1, 0),
-					createBuffer(0), createBuffer(0), createBuffer(0), createBuffer(0),
-					createBarrier(2, 0), createBarrier(3, 0),
-					createBuffer(0), createBuffer(0),
-					createBarrier(4, 0), createBarrier(5, 0), createBarrier(6, 0),
-					createBuffer(0), createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-			buffer.registerCheckpointEventHandler(handler);
-			handler.setNextExpectedCheckpointId(1L);
-			
-			for (BufferOrEvent boe : sequence) {
-				if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-					assertEquals(boe, buffer.getNextNonBlocked());
-				}
-			}
-
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Validates that the buffer correctly aligns the streams for inputs with
-	 * multiple input channels, by buffering and blocking certain inputs.
-	 */
-	@Test
-	public void testMultiChannelWithBarriers() {
-		try {
-			BufferOrEvent[] sequence = {
-					// checkpoint with blocked data
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(1, 1), createBarrier(1, 2),
-					createBuffer(2), createBuffer(1), createBuffer(0),
-					createBarrier(1, 0),
-					
-					// checkpoint without blocked data
-					createBuffer(0), createBuffer(0), createBuffer(1), createBuffer(1), createBuffer(2),
-					createBarrier(2, 0), createBarrier(2, 1), createBarrier(2, 2),
-					
-					// checkpoint with data only from one channel
-					createBuffer(2), createBuffer(2),
-					createBarrier(3, 2),
-					createBuffer(2), createBuffer(2),
-					createBarrier(3, 0), createBarrier(3, 1),
-					
-					// empty checkpoint
-					createBarrier(4, 1), createBarrier(4, 2), createBarrier(4, 0),
-
-					// checkpoint with blocked data in mixed order
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(5, 1),
-					createBuffer(2), createBuffer(0), createBuffer(2), createBuffer(1),
-					createBarrier(5, 2),
-					createBuffer(1), createBuffer(0), createBuffer(2), createBuffer(1),
-					createBarrier(5, 0),
-					
-					// some trailing data
-					createBuffer(0),
-					createEndOfPartition(0), createEndOfPartition(1), createEndOfPartition(2)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-			buffer.registerCheckpointEventHandler(handler);
-			handler.setNextExpectedCheckpointId(1L);
-			
-			// pre checkpoint 1
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			assertEquals(1L, handler.getNextExpectedCheckpointId());
-
-			// blocking while aligning for checkpoint 1
-			check(sequence[7], buffer.getNextNonBlocked());
-			assertEquals(1L, handler.getNextExpectedCheckpointId());
-
-			// checkpoint 1 done, returning buffered data
-			check(sequence[5], buffer.getNextNonBlocked());
-			assertEquals(2L, handler.getNextExpectedCheckpointId());
-			check(sequence[6], buffer.getNextNonBlocked());
-
-			// pre checkpoint 2
-			check(sequence[9], buffer.getNextNonBlocked());
-			check(sequence[10], buffer.getNextNonBlocked());
-			check(sequence[11], buffer.getNextNonBlocked());
-			check(sequence[12], buffer.getNextNonBlocked());
-			check(sequence[13], buffer.getNextNonBlocked());
-			assertEquals(2L, handler.getNextExpectedCheckpointId());
-			
-			// checkpoint 2 barriers come together
-			check(sequence[17], buffer.getNextNonBlocked());
-			assertEquals(3L, handler.getNextExpectedCheckpointId());
-			check(sequence[18], buffer.getNextNonBlocked());
-
-			// checkpoint 3 starts, data buffered
-			check(sequence[20], buffer.getNextNonBlocked());
-			assertEquals(4L, handler.getNextExpectedCheckpointId());
-			check(sequence[21], buffer.getNextNonBlocked());
-
-			// checkpoint 4 happens without extra data
-			
-			// pre checkpoint 5
-			check(sequence[27], buffer.getNextNonBlocked());
-			assertEquals(5L, handler.getNextExpectedCheckpointId());
-			check(sequence[28], buffer.getNextNonBlocked());
-			check(sequence[29], buffer.getNextNonBlocked());
-			
-			// checkpoint 5 aligning
-			check(sequence[31], buffer.getNextNonBlocked());
-			check(sequence[32], buffer.getNextNonBlocked());
-			check(sequence[33], buffer.getNextNonBlocked());
-			check(sequence[37], buffer.getNextNonBlocked());
-			
-			// buffered data from checkpoint 5 alignment
-			check(sequence[34], buffer.getNextNonBlocked());
-			check(sequence[36], buffer.getNextNonBlocked());
-			check(sequence[38], buffer.getNextNonBlocked());
-			check(sequence[39], buffer.getNextNonBlocked());
-			
-			// remaining data
-			check(sequence[41], buffer.getNextNonBlocked());
-			check(sequence[42], buffer.getNextNonBlocked());
-			check(sequence[43], buffer.getNextNonBlocked());
-			check(sequence[44], buffer.getNextNonBlocked());
-			
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testMultiChannelTrailingBlockedData() {
-		try {
-			BufferOrEvent[] sequence = {
-					createBuffer(0), createBuffer(1), createBuffer(2),
-					createBarrier(1, 1), createBarrier(1, 2), createBarrier(1, 0),
-					
-					createBuffer(2), createBuffer(1), createBuffer(0),
-					createBarrier(2, 1),
-					createBuffer(1), createBuffer(1), createEndOfPartition(1), createBuffer(0), createBuffer(2),
-					createBarrier(2, 2),
-					createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-			buffer.registerCheckpointEventHandler(handler);
-			handler.setNextExpectedCheckpointId(1L);
-
-			// pre-checkpoint 1
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			assertEquals(1L, handler.getNextExpectedCheckpointId());
-
-			// pre-checkpoint 2
-			check(sequence[6], buffer.getNextNonBlocked());
-			assertEquals(2L, handler.getNextExpectedCheckpointId());
-			check(sequence[7], buffer.getNextNonBlocked());
-			check(sequence[8], buffer.getNextNonBlocked());
-			
-			// checkpoint 2 alignment
-			check(sequence[13], buffer.getNextNonBlocked());
-			check(sequence[14], buffer.getNextNonBlocked());
-			check(sequence[18], buffer.getNextNonBlocked());
-			check(sequence[19], buffer.getNextNonBlocked());
-
-			// end of stream: remaining buffered contents
-			check(sequence[10], buffer.getNextNonBlocked());
-			check(sequence[11], buffer.getNextNonBlocked());
-			check(sequence[12], buffer.getNextNonBlocked());
-			check(sequence[16], buffer.getNextNonBlocked());
-			check(sequence[17], buffer.getNextNonBlocked());
-
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-			
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Validates that the buffer correctly aligns the streams in cases
-	 * where some channels receive barriers from multiple successive checkpoints
-	 * before the pending checkpoint is complete.
-	 */
-	@Test
-	public void testMultiChannelWithQueuedFutureBarriers() {
-		try {
-			BufferOrEvent[] sequence = {
-					// checkpoint 1 - with blocked data
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(1, 1), createBarrier(1, 2),
-					createBuffer(2), createBuffer(1), createBuffer(0),
-					createBarrier(1, 0),
-					createBuffer(1), createBuffer(0),
-
-					// checkpoint 2 - where future checkpoint barriers come before
-					// the current checkpoint is complete
-					createBarrier(2, 1),
-					createBuffer(1), createBuffer(2), createBarrier(2, 0),
-					createBarrier(3, 0), createBuffer(0),
-					createBarrier(3, 1), createBuffer(0), createBuffer(1), createBuffer(2),
-					createBarrier(4, 1), createBuffer(1), createBuffer(2),
-
-					// complete checkpoint 2, send a barrier for checkpoints 4 and 5
-					createBarrier(2, 2),
-					createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0),
-					createBarrier(4, 0),
-					createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0),
-					createBarrier(5, 1),
-
-					// complete checkpoint 3
-					createBarrier(3, 2),
-					createBuffer(2), createBuffer(1), createBuffer(2), createBuffer(0),
-					createBarrier(6, 1),
-					
-					// complete checkpoint 4, checkpoint 5 remains not fully triggered
-					createBarrier(4, 2),
-					createBuffer(2),
-					createBuffer(1), createEndOfPartition(1),
-					createBuffer(2), createEndOfPartition(2),
-					createBuffer(0), createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-			buffer.registerCheckpointEventHandler(handler);
-			handler.setNextExpectedCheckpointId(1L);
-
-			// around checkpoint 1
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			check(sequence[7], buffer.getNextNonBlocked());
-			
-			check(sequence[5], buffer.getNextNonBlocked());
-			assertEquals(2L, handler.getNextExpectedCheckpointId());
-			check(sequence[6], buffer.getNextNonBlocked());
-			check(sequence[9], buffer.getNextNonBlocked());
-			check(sequence[10], buffer.getNextNonBlocked());
-
-			// alignment of checkpoint 2 - buffering also some barriers for
-			// checkpoints 3 and 4
-			check(sequence[13], buffer.getNextNonBlocked());
-			check(sequence[20], buffer.getNextNonBlocked());
-			check(sequence[23], buffer.getNextNonBlocked());
-			
-			// checkpoint 2 completed
-			check(sequence[12], buffer.getNextNonBlocked());
-			check(sequence[25], buffer.getNextNonBlocked());
-			check(sequence[27], buffer.getNextNonBlocked());
-			check(sequence[30], buffer.getNextNonBlocked());
-			check(sequence[32], buffer.getNextNonBlocked());
-
-			// checkpoint 3 completed (emit buffered)
-			check(sequence[16], buffer.getNextNonBlocked());
-			check(sequence[18], buffer.getNextNonBlocked());
-			check(sequence[19], buffer.getNextNonBlocked());
-			check(sequence[28], buffer.getNextNonBlocked());
-			
-			// past checkpoint 3
-			check(sequence[36], buffer.getNextNonBlocked());
-			check(sequence[38], buffer.getNextNonBlocked());
-
-			// checkpoint 4 completed (emit buffered)
-			check(sequence[22], buffer.getNextNonBlocked());
-			check(sequence[26], buffer.getNextNonBlocked());
-			check(sequence[31], buffer.getNextNonBlocked());
-			check(sequence[33], buffer.getNextNonBlocked());
-			check(sequence[39], buffer.getNextNonBlocked());
-			
-			// past checkpoint 4, alignment for checkpoint 5
-			check(sequence[42], buffer.getNextNonBlocked());
-			check(sequence[45], buffer.getNextNonBlocked());
-			check(sequence[46], buffer.getNextNonBlocked());
-			
-			// abort checkpoint 5 (end of partition)
-			check(sequence[37], buffer.getNextNonBlocked());
-			
-			// start checkpoint 6 alignment
-			check(sequence[47], buffer.getNextNonBlocked());
-			check(sequence[48], buffer.getNextNonBlocked());
-			
-			// end of input, emit remainder
-			check(sequence[43], buffer.getNextNonBlocked());
-			check(sequence[44], buffer.getNextNonBlocked());
-			
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Validates that the buffer skips over the current checkpoint if it
-	 * receives a barrier from a later checkpoint on a non-blocked input.
-	 */
-	@Test
-	public void testMultiChannelSkippingCheckpoints() {
-		try {
-			BufferOrEvent[] sequence = {
-					// checkpoint 1 - with blocked data
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(1, 1), createBarrier(1, 2),
-					createBuffer(2), createBuffer(1), createBuffer(0),
-					createBarrier(1, 0),
-					createBuffer(1), createBuffer(0),
-
-					// checkpoint 2 will not complete: pre-mature barrier from checkpoint 3
-					createBarrier(2, 1),
-					createBuffer(1), createBuffer(2),
-					createBarrier(2, 0),
-					createBuffer(2), createBuffer(0),
-					createBarrier(3, 2),
-					
-					createBuffer(2),
-					createBuffer(1), createEndOfPartition(1),
-					createBuffer(2), createEndOfPartition(2),
-					createBuffer(0), createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-			buffer.registerCheckpointEventHandler(handler);
-			handler.setNextExpectedCheckpointId(1L);
-
-			// checkpoint 1
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			check(sequence[7], buffer.getNextNonBlocked());
-			assertEquals(1L, buffer.getCurrentCheckpointId());
-			
-			check(sequence[5], buffer.getNextNonBlocked());
-			check(sequence[6], buffer.getNextNonBlocked());
-			check(sequence[9], buffer.getNextNonBlocked());
-			check(sequence[10], buffer.getNextNonBlocked());
-
-			// alignment of checkpoint 2
-			check(sequence[13], buffer.getNextNonBlocked());
-			assertEquals(2L, buffer.getCurrentCheckpointId());
-			check(sequence[15], buffer.getNextNonBlocked());
-
-			// checkpoint 2 aborted, checkpoint 3 started
-			check(sequence[12], buffer.getNextNonBlocked());
-			assertEquals(3L, buffer.getCurrentCheckpointId());
-			check(sequence[16], buffer.getNextNonBlocked());
-			check(sequence[19], buffer.getNextNonBlocked());
-			check(sequence[20], buffer.getNextNonBlocked());
-			
-			// checkpoint 3 aborted (end of partition)
-			check(sequence[18], buffer.getNextNonBlocked());
-			check(sequence[21], buffer.getNextNonBlocked());
-			check(sequence[22], buffer.getNextNonBlocked());
-			check(sequence[23], buffer.getNextNonBlocked());
-			check(sequence[24], buffer.getNextNonBlocked());
-
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-			
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Validates that the buffer skips over the current checkpoint if it
-	 * receives a barrier from a later checkpoint on a non-blocked input.
-	 */
-	@Test
-	public void testMultiChannelJumpingOverCheckpoint() {
-		try {
-			BufferOrEvent[] sequence = {
-					// checkpoint 1 - with blocked data
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(1, 1), createBarrier(1, 2),
-					createBuffer(2), createBuffer(1), createBuffer(0),
-					createBarrier(1, 0),
-					createBuffer(1), createBuffer(0),
-
-					// checkpoint 2 will not complete: pre-mature barrier from checkpoint 3
-					createBarrier(2, 1),
-					createBuffer(1), createBuffer(2),
-					createBarrier(2, 0),
-					createBuffer(2), createBuffer(0),
-					createBarrier(3, 1),
-					createBuffer(1), createBuffer(2),
-					createBarrier(3, 0),
-					createBuffer(2), createBuffer(0),
-					createBarrier(4, 2),
-
-					createBuffer(2),
-					createBuffer(1), createEndOfPartition(1),
-					createBuffer(2), createEndOfPartition(2),
-					createBuffer(0), createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-			buffer.registerCheckpointEventHandler(handler);
-			handler.setNextExpectedCheckpointId(1L);
-
-			// checkpoint 1
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			check(sequence[7], buffer.getNextNonBlocked());
-			assertEquals(1L, buffer.getCurrentCheckpointId());
-
-			check(sequence[5], buffer.getNextNonBlocked());
-			check(sequence[6], buffer.getNextNonBlocked());
-			check(sequence[9], buffer.getNextNonBlocked());
-			check(sequence[10], buffer.getNextNonBlocked());
-
-			// alignment of checkpoint 2
-			check(sequence[13], buffer.getNextNonBlocked());
-			assertEquals(2L, buffer.getCurrentCheckpointId());
-			check(sequence[15], buffer.getNextNonBlocked());
-			check(sequence[19], buffer.getNextNonBlocked());
-			check(sequence[21], buffer.getNextNonBlocked());
-
-			// checkpoint 2 aborted, checkpoint 4 started. replay buffered
-			check(sequence[12], buffer.getNextNonBlocked());
-			assertEquals(4L, buffer.getCurrentCheckpointId());
-			check(sequence[16], buffer.getNextNonBlocked());
-			check(sequence[18], buffer.getNextNonBlocked());
-			check(sequence[22], buffer.getNextNonBlocked());
-			
-			// align checkpoint 4 remainder
-			check(sequence[25], buffer.getNextNonBlocked());
-			check(sequence[26], buffer.getNextNonBlocked());
-			
-			// checkpoint 4 aborted (due to end of partition)
-			check(sequence[24], buffer.getNextNonBlocked());
-			check(sequence[27], buffer.getNextNonBlocked());
-			check(sequence[28], buffer.getNextNonBlocked());
-			check(sequence[29], buffer.getNextNonBlocked());
-			check(sequence[30], buffer.getNextNonBlocked());
-
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * Validates that the buffer skips over a later checkpoint if it
-	 * receives a barrier from an even later checkpoint on a blocked input.
-	 */
-	@Test
-	public void testMultiChannelSkippingCheckpointsViaBlockedInputs() {
-		try {
-			BufferOrEvent[] sequence = {
-					// checkpoint 1 - with blocked data
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(1, 1), createBarrier(1, 2),
-					createBuffer(2), createBuffer(1), createBuffer(0),
-					createBarrier(1, 0),
-					createBuffer(1), createBuffer(0),
-
-					// checkpoint 2 will not complete: pre-mature barrier from checkpoint 3
-					createBarrier(2, 1),
-					createBuffer(1), createBuffer(2),
-					createBarrier(2, 0),
-					createBuffer(1), createBuffer(0),
-
-					createBarrier(3, 0), // queued barrier on blocked input
-					createBuffer(0),
-					
-					createBarrier(4, 1), // pre-mature barrier on blocked input
-					createBuffer(1),
-					createBuffer(0),
-					createBuffer(2),
-
-					// complete checkpoint 2
-					createBarrier(2, 2),
-					createBuffer(0),
-					
-					createBarrier(3, 2), // should be ignored
-					createBuffer(2),
-					createBarrier(4, 0),
-					createBuffer(0), createBuffer(1), createBuffer(2),
-					createBarrier(4, 2),
-					
-					createBuffer(1), createEndOfPartition(1),
-					createBuffer(2), createEndOfPartition(2),
-					createBuffer(0), createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			// checkpoint 1
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			check(sequence[7], buffer.getNextNonBlocked());
-			assertEquals(1L, buffer.getCurrentCheckpointId());
-			check(sequence[5], buffer.getNextNonBlocked());
-			check(sequence[6], buffer.getNextNonBlocked());
-			check(sequence[9], buffer.getNextNonBlocked());
-			check(sequence[10], buffer.getNextNonBlocked());
-
-			// alignment of checkpoint 2
-			check(sequence[13], buffer.getNextNonBlocked());
-			check(sequence[22], buffer.getNextNonBlocked());
-			assertEquals(2L, buffer.getCurrentCheckpointId());
-
-			// checkpoint 2 completed
-			check(sequence[12], buffer.getNextNonBlocked());
-			check(sequence[15], buffer.getNextNonBlocked());
-			check(sequence[16], buffer.getNextNonBlocked());
-			
-			// checkpoint 3 skipped, alignment for 4 started
-			check(sequence[18], buffer.getNextNonBlocked());
-			assertEquals(4L, buffer.getCurrentCheckpointId());
-			check(sequence[21], buffer.getNextNonBlocked());
-			check(sequence[24], buffer.getNextNonBlocked());
-			check(sequence[26], buffer.getNextNonBlocked());
-			check(sequence[30], buffer.getNextNonBlocked());
-			
-			// checkpoint 4 completed
-			check(sequence[20], buffer.getNextNonBlocked());
-			check(sequence[28], buffer.getNextNonBlocked());
-			check(sequence[29], buffer.getNextNonBlocked());
-			
-			check(sequence[32], buffer.getNextNonBlocked());
-			check(sequence[33], buffer.getNextNonBlocked());
-			check(sequence[34], buffer.getNextNonBlocked());
-			check(sequence[35], buffer.getNextNonBlocked());
-			check(sequence[36], buffer.getNextNonBlocked());
-			check(sequence[37], buffer.getNextNonBlocked());
-			
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testEarlyCleanup() {
-		try {
-			BufferOrEvent[] sequence = {
-					createBuffer(0), createBuffer(1), createBuffer(2),
-					createBarrier(1, 1), createBarrier(1, 2), createBarrier(1, 0),
-
-					createBuffer(2), createBuffer(1), createBuffer(0),
-					createBarrier(2, 1),
-					createBuffer(1), createBuffer(1), createEndOfPartition(1), createBuffer(0), createBuffer(2),
-					createBarrier(2, 2),
-					createBuffer(2), createEndOfPartition(2), createBuffer(0), createEndOfPartition(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-
-			ValidatingCheckpointHandler handler = new ValidatingCheckpointHandler();
-			buffer.registerCheckpointEventHandler(handler);
-			handler.setNextExpectedCheckpointId(1L);
-
-			// pre-checkpoint 1
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			assertEquals(1L, handler.getNextExpectedCheckpointId());
-
-			// pre-checkpoint 2
-			check(sequence[6], buffer.getNextNonBlocked());
-			assertEquals(2L, handler.getNextExpectedCheckpointId());
-			check(sequence[7], buffer.getNextNonBlocked());
-			check(sequence[8], buffer.getNextNonBlocked());
-
-			// checkpoint 2 alignment
-			check(sequence[13], buffer.getNextNonBlocked());
-			check(sequence[14], buffer.getNextNonBlocked());
-			check(sequence[18], buffer.getNextNonBlocked());
-			check(sequence[19], buffer.getNextNonBlocked());
-
-			// end of stream: remaining buffered contents
-			buffer.getNextNonBlocked();
-			buffer.cleanup();
-
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testStartAlignmentWithClosedChannels() {
-		try {
-			BufferOrEvent[] sequence = {
-					// close some channels immediately 
-					createEndOfPartition(2), createEndOfPartition(1),
-
-					// checkpoint without blocked data
-					createBuffer(0), createBuffer(0), createBuffer(3),
-					createBarrier(2, 3), createBarrier(2, 0),
-
-					// checkpoint with blocked data
-					createBuffer(3), createBuffer(0),
-					createBarrier(3, 3),
-					createBuffer(3), createBuffer(0),
-					createBarrier(3, 0),
-
-					// empty checkpoint
-					createBarrier(4, 0), createBarrier(4, 3),
-					
-					// some data, one channel closes
-					createBuffer(0), createBuffer(0), createBuffer(3),
-					createEndOfPartition(0),
-					
-					// checkpoint on last remaining channel
-					createBuffer(3),
-					createBarrier(5, 3),
-					createBuffer(3),
-					createEndOfPartition(3)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 4, Arrays.asList(sequence));
-			
-			BarrierBuffer buffer = new BarrierBuffer(gate, IO_MANAGER);
-			
-			// pre checkpoint 2
-			check(sequence[0], buffer.getNextNonBlocked());
-			check(sequence[1], buffer.getNextNonBlocked());
-			check(sequence[2], buffer.getNextNonBlocked());
-			check(sequence[3], buffer.getNextNonBlocked());
-			check(sequence[4], buffer.getNextNonBlocked());
-
-			// checkpoint 3 alignment
-			check(sequence[7], buffer.getNextNonBlocked());
-			assertEquals(2L, buffer.getCurrentCheckpointId());
-			check(sequence[8], buffer.getNextNonBlocked());
-			check(sequence[11], buffer.getNextNonBlocked());
-
-			// checkpoint 3 buffered
-			check(sequence[10], buffer.getNextNonBlocked());
-			assertEquals(3L, buffer.getCurrentCheckpointId());
-
-			// after checkpoint 4
-			check(sequence[15], buffer.getNextNonBlocked());
-			assertEquals(4L, buffer.getCurrentCheckpointId());
-			check(sequence[16], buffer.getNextNonBlocked());
-			check(sequence[17], buffer.getNextNonBlocked());
-			check(sequence[18], buffer.getNextNonBlocked());
-
-			check(sequence[19], buffer.getNextNonBlocked());
-			check(sequence[21], buffer.getNextNonBlocked());
-			assertEquals(5L, buffer.getCurrentCheckpointId());
-			check(sequence[22], buffer.getNextNonBlocked());
-
-			assertNull(buffer.getNextNonBlocked());
-			assertNull(buffer.getNextNonBlocked());
-			
-			buffer.cleanup();
-			
-			checkNoTempFilesRemain();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testEndOfStreamWhileCheckpoint() {
-		
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utils
-	// ------------------------------------------------------------------------
-
-	private static BufferOrEvent createBarrier(long id, int channel) {
-		return new BufferOrEvent(new CheckpointBarrier(id, System.currentTimeMillis()), channel);
-	}
-
-	private static BufferOrEvent createBuffer(int channel) {
-		// since we have no access to the contents, we need to use the size as an
-		// identifier to validate correctness here
-		Buffer buf = new Buffer(
-				MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE),
-				FreeingBufferRecycler.INSTANCE);
-		
-		buf.setSize(SIZE_COUNTER++);
-		return new BufferOrEvent(buf, channel);
-	}
-
-	private static BufferOrEvent createEndOfPartition(int channel) {
-		return new BufferOrEvent(EndOfPartitionEvent.INSTANCE, channel);
-	}
-	
-	private static void check(BufferOrEvent expected, BufferOrEvent present) {
-		assertNotNull(expected);
-		assertNotNull(present);
-		assertEquals(expected.isBuffer(), present.isBuffer());
-		
-		if (expected.isBuffer()) {
-			// since we have no access to the contents, we need to use the size as an
-			// identifier to validate correctness here
-			assertEquals(expected.getBuffer().getSize(), present.getBuffer().getSize());
-		}
-		else {
-			assertEquals(expected.getEvent(), present.getEvent());
-		}
-	}
-	
-	private static void checkNoTempFilesRemain() {
-		// validate that all temp files have been removed
-		for (File dir : IO_MANAGER.getSpillingDirectories()) {
-			for (String file : dir.list()) {
-				if (file != null && !(file.equals(".") || file.equals(".."))) {
-					fail("barrier buffer did not clean up temp files. remaining file: " + file);
-				}
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Testing Mocks
-	// ------------------------------------------------------------------------
-
-	private static class ValidatingCheckpointHandler implements EventListener<CheckpointBarrier> {
-		
-		private long nextExpectedCheckpointId = -1L;
-
-		public void setNextExpectedCheckpointId(long nextExpectedCheckpointId) {
-			this.nextExpectedCheckpointId = nextExpectedCheckpointId;
-		}
-
-		public long getNextExpectedCheckpointId() {
-			return nextExpectedCheckpointId;
-		}
-
-		@Override
-		public void onEvent(CheckpointBarrier barrier) {
-			assertNotNull(barrier);
-			assertTrue("wrong checkpoint id", nextExpectedCheckpointId == -1L || nextExpectedCheckpointId == barrier.getId());
-			assertTrue(barrier.getTimestamp() > 0);
-			nextExpectedCheckpointId++;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
deleted file mode 100644
index b9b6e5f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ /dev/null
@@ -1,367 +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.io;
-
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import org.junit.Test;
-
-import java.util.Arrays;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests for the behavior of the barrier tracker.
- */
-public class BarrierTrackerTest {
-	
-	private static final int PAGE_SIZE = 512;
-	
-	@Test
-	public void testSingleChannelNoBarriers() {
-		try {
-			BufferOrEvent[] sequence = { createBuffer(0), createBuffer(0), createBuffer(0) };
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence));
-			BarrierTracker tracker = new BarrierTracker(gate);
-
-			for (BufferOrEvent boe : sequence) {
-				assertEquals(boe, tracker.getNextNonBlocked());
-			}
-			
-			assertNull(tracker.getNextNonBlocked());
-			assertNull(tracker.getNextNonBlocked());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testMultiChannelNoBarriers() {
-		try {
-			BufferOrEvent[] sequence = { createBuffer(2), createBuffer(2), createBuffer(0),
-					createBuffer(1), createBuffer(0), createBuffer(3),
-					createBuffer(1), createBuffer(1), createBuffer(2)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 4, Arrays.asList(sequence));
-			BarrierTracker tracker = new BarrierTracker(gate);
-
-			for (BufferOrEvent boe : sequence) {
-				assertEquals(boe, tracker.getNextNonBlocked());
-			}
-
-			assertNull(tracker.getNextNonBlocked());
-			assertNull(tracker.getNextNonBlocked());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSingleChannelWithBarriers() {
-		try {
-			BufferOrEvent[] sequence = {
-					createBuffer(0), createBuffer(0), createBuffer(0),
-					createBarrier(1, 0),
-					createBuffer(0), createBuffer(0), createBuffer(0), createBuffer(0),
-					createBarrier(2, 0), createBarrier(3, 0),
-					createBuffer(0), createBuffer(0),
-					createBarrier(4, 0), createBarrier(5, 0), createBarrier(6, 0),
-					createBuffer(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence));
-			BarrierTracker tracker = new BarrierTracker(gate);
-
-			CheckpointSequenceValidator validator =
-					new CheckpointSequenceValidator(1, 2, 3, 4, 5, 6);
-			tracker.registerCheckpointEventHandler(validator);
-			
-			for (BufferOrEvent boe : sequence) {
-				if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-					assertEquals(boe, tracker.getNextNonBlocked());
-				}
-			}
-
-			assertNull(tracker.getNextNonBlocked());
-			assertNull(tracker.getNextNonBlocked());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSingleChannelWithSkippedBarriers() {
-		try {
-			BufferOrEvent[] sequence = {
-					createBuffer(0),
-					createBarrier(1, 0),
-					createBuffer(0), createBuffer(0),
-					createBarrier(3, 0), createBuffer(0),
-					createBarrier(4, 0), createBarrier(6, 0), createBuffer(0),
-					createBarrier(7, 0), createBuffer(0), createBarrier(10, 0),
-					createBuffer(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 1, Arrays.asList(sequence));
-			BarrierTracker tracker = new BarrierTracker(gate);
-
-			CheckpointSequenceValidator validator =
-					new CheckpointSequenceValidator(1, 3, 4, 6, 7, 10);
-			tracker.registerCheckpointEventHandler(validator);
-
-			for (BufferOrEvent boe : sequence) {
-				if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-					assertEquals(boe, tracker.getNextNonBlocked());
-				}
-			}
-
-			assertNull(tracker.getNextNonBlocked());
-			assertNull(tracker.getNextNonBlocked());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testMultiChannelWithBarriers() {
-		try {
-			BufferOrEvent[] sequence = {
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(1, 1), createBarrier(1, 2),
-					createBuffer(2), createBuffer(1),
-					createBarrier(1, 0),
-					
-					createBuffer(0), createBuffer(0), createBuffer(1), createBuffer(1), createBuffer(2),
-					createBarrier(2, 0), createBarrier(2, 1), createBarrier(2, 2),
-					
-					createBuffer(2), createBuffer(2),
-					createBarrier(3, 2),
-					createBuffer(2), createBuffer(2),
-					createBarrier(3, 0), createBarrier(3, 1),
-					
-					createBarrier(4, 1), createBarrier(4, 2), createBarrier(4, 0),
-					
-					createBuffer(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierTracker tracker = new BarrierTracker(gate);
-
-			CheckpointSequenceValidator validator =
-					new CheckpointSequenceValidator(1, 2, 3, 4);
-			tracker.registerCheckpointEventHandler(validator);
-
-			for (BufferOrEvent boe : sequence) {
-				if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-					assertEquals(boe, tracker.getNextNonBlocked());
-				}
-			}
-
-			assertNull(tracker.getNextNonBlocked());
-			assertNull(tracker.getNextNonBlocked());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testMultiChannelSkippingCheckpoints() {
-		try {
-			BufferOrEvent[] sequence = {
-					createBuffer(0), createBuffer(2), createBuffer(0),
-					createBarrier(1, 1), createBarrier(1, 2),
-					createBuffer(2), createBuffer(1),
-					createBarrier(1, 0),
-
-					createBuffer(0), createBuffer(0), createBuffer(1), createBuffer(1), createBuffer(2),
-					createBarrier(2, 0), createBarrier(2, 1), createBarrier(2, 2),
-
-					createBuffer(2), createBuffer(2),
-					createBarrier(3, 2),
-					createBuffer(2), createBuffer(2),
-					
-					// jump to checkpoint 4
-					createBarrier(4, 0),
-					createBuffer(0), createBuffer(1), createBuffer(2),
-					createBarrier(4, 1),
-					createBuffer(1),
-					createBarrier(4, 2),
-					
-					createBuffer(0)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierTracker tracker = new BarrierTracker(gate);
-
-			CheckpointSequenceValidator validator =
-					new CheckpointSequenceValidator(1, 2, 4);
-			tracker.registerCheckpointEventHandler(validator);
-
-			for (BufferOrEvent boe : sequence) {
-				if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-					assertEquals(boe, tracker.getNextNonBlocked());
-				}
-			}
-			
-			assertNull(tracker.getNextNonBlocked());
-			assertNull(tracker.getNextNonBlocked());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	/**
-	 * This test validates that the barrier tracker does not immediately
-	 * discard a pending checkpoint as soon as it sees a barrier from a
-	 * later checkpoint from some channel.
-	 * 
-	 * This behavior is crucial, otherwise topologies where different inputs
-	 * have different latency (and that latency is close to or higher than the
-	 * checkpoint interval) may skip many checkpoints, or fail to complete a
-	 * checkpoint all together.
-	 */
-	@Test
-	public void testCompleteCheckpointsOnLateBarriers() {
-		try {
-			BufferOrEvent[] sequence = {
-					// checkpoint 2
-					createBuffer(1), createBuffer(1), createBuffer(0), createBuffer(2),
-					createBarrier(2, 1), createBarrier(2, 0), createBarrier(2, 2),
-					
-					// incomplete checkpoint 3
-					createBuffer(1), createBuffer(0),
-					createBarrier(3, 1), createBarrier(3, 2),
-					
-					// some barriers from checkpoint 4
-					createBuffer(1), createBuffer(0),
-					createBarrier(4, 2), createBarrier(4, 1),
-					createBuffer(1), createBuffer(2),
-	
-					// last barrier from checkpoint 3
-					createBarrier(3, 0),
-					
-					// complete checkpoint 4
-					createBuffer(0), createBarrier(4, 0),
-					
-					// regular checkpoint 5
-					createBuffer(1), createBuffer(2), createBarrier(5, 1), 
-					createBuffer(0), createBarrier(5, 0),
-					createBuffer(1), createBarrier(5, 2),
-					
-					// checkpoint 6 (incomplete),
-					createBuffer(1), createBarrier(6, 1),
-					createBuffer(0), createBarrier(6, 0),
-					
-					// checkpoint 7, with early barriers for checkpoints 8 and 9
-					createBuffer(1), createBarrier(7, 1),
-					createBuffer(0), createBarrier(7, 2),
-					createBuffer(2), createBarrier(8, 2), 
-					createBuffer(0), createBarrier(8, 1),
-					createBuffer(1), createBarrier(9, 1),
-					
-					// complete checkpoint 7, first barriers from checkpoint 10
-					createBarrier(7, 0),
-					createBuffer(0), createBarrier(9, 2),
-					createBuffer(2), createBarrier(10, 2),
-					
-					// complete checkpoint 8 and 9
-					createBarrier(8, 0),
-					createBuffer(1), createBuffer(2), createBarrier(9, 0),
-					
-					// trailing data
-					createBuffer(1), createBuffer(0), createBuffer(2)
-			};
-
-			MockInputGate gate = new MockInputGate(PAGE_SIZE, 3, Arrays.asList(sequence));
-			BarrierTracker tracker = new BarrierTracker(gate);
-
-			CheckpointSequenceValidator validator =
-					new CheckpointSequenceValidator(2, 3, 4, 5, 7, 8, 9);
-			tracker.registerCheckpointEventHandler(validator);
-
-			for (BufferOrEvent boe : sequence) {
-				if (boe.isBuffer() || boe.getEvent().getClass() != CheckpointBarrier.class) {
-					assertEquals(boe, tracker.getNextNonBlocked());
-				}
-			}
-
-			assertNull(tracker.getNextNonBlocked());
-			assertNull(tracker.getNextNonBlocked());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utils
-	// ------------------------------------------------------------------------
-
-	private static BufferOrEvent createBarrier(long id, int channel) {
-		return new BufferOrEvent(new CheckpointBarrier(id, System.currentTimeMillis()), channel);
-	}
-
-	private static BufferOrEvent createBuffer(int channel) {
-		return new BufferOrEvent(
-				new Buffer(MemorySegmentFactory.wrap(new byte[]{1, 2}), FreeingBufferRecycler.INSTANCE), channel);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Testing Mocks
-	// ------------------------------------------------------------------------
-	
-	private static class CheckpointSequenceValidator implements EventListener<CheckpointBarrier> {
-
-		private final long[] checkpointIDs;
-		
-		private int i = 0;
-
-		private CheckpointSequenceValidator(long... checkpointIDs) {
-			this.checkpointIDs = checkpointIDs;
-		}
-		
-		@Override
-		public void onEvent(CheckpointBarrier barrier) {
-			assertTrue("More checkpoints than expected", i < checkpointIDs.length);
-			assertNotNull(barrier);
-			assertEquals("wrong checkpoint id", checkpointIDs[i++], barrier.getId());
-			assertTrue(barrier.getTimestamp() > 0);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
deleted file mode 100644
index e85eddb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
+++ /dev/null
@@ -1,407 +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.io;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Random;
-
-
-import static org.junit.Assert.*;
-
-public class BufferSpillerTest {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(BufferSpillerTest.class);
-
-	private static final int PAGE_SIZE = 4096;
-
-	private static IOManager IO_MANAGER;
-
-	private BufferSpiller spiller;
-
-
-	// ------------------------------------------------------------------------
-	//  Setup / Cleanup
-	// ------------------------------------------------------------------------
-	
-	@BeforeClass
-	public static void setupIOManager() {
-		IO_MANAGER = new IOManagerAsync();
-	}
-
-	@AfterClass
-	public static void shutdownIOManager() {
-		IO_MANAGER.shutdown();
-	}
-	
-	@Before
-	public void createSpiller() {
-		try {
-			spiller = new BufferSpiller(IO_MANAGER, PAGE_SIZE);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail("Cannot create BufferSpiller: " + e.getMessage());
-		}
-	}
-	
-	@After
-	public void cleanupSpiller() {
-		if (spiller != null) {
-			try {
-				spiller.close();
-			}
-			catch (Exception e) {
-				e.printStackTrace();
-				fail("Cannot properly close the BufferSpiller: " + e.getMessage());
-			}
-			
-			assertFalse(spiller.getCurrentChannel().isOpen());
-			assertFalse(spiller.getCurrentSpillFile().exists());
-		}
-		
-		checkNoTempFilesRemain();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Tests
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void testRollOverEmptySequences() {
-		try {
-			assertNull(spiller.rollOver());
-			assertNull(spiller.rollOver());
-			assertNull(spiller.rollOver());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSpillAndRollOverSimple() {
-		try {
-			final Random rnd = new Random();
-			final Random bufferRnd = new Random();
-
-			final int maxNumEventsAndBuffers = 3000;
-			final int maxNumChannels = 1656;
-
-			// do multiple spilling / rolling over rounds
-			for (int round = 0; round < 5; round++) {
-				
-				final long bufferSeed = rnd.nextLong();
-				bufferRnd.setSeed(bufferSeed);
-				
-				final int numEventsAndBuffers = rnd.nextInt(maxNumEventsAndBuffers) + 1;
-				final int numChannels = rnd.nextInt(maxNumChannels) + 1;
-				
-				final ArrayList<BufferOrEvent> events = new ArrayList<BufferOrEvent>(128);
-
-				// generate sequence
-				for (int i = 0; i < numEventsAndBuffers; i++) {
-					boolean isEvent = rnd.nextDouble() < 0.05d;
-					if (isEvent) {
-						BufferOrEvent evt = generateRandomEvent(rnd, numChannels);
-						events.add(evt);
-						spiller.add(evt);
-					}
-					else {
-						BufferOrEvent evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels));
-						spiller.add(evt);
-					}
-				}
-
-				// reset and create reader
-				bufferRnd.setSeed(bufferSeed);
-			
-				BufferSpiller.SpilledBufferOrEventSequence seq = spiller.rollOver();
-				seq.open();
-
-				// read and validate the sequence
-
-				int numEvent = 0;
-				for (int i = 0; i < numEventsAndBuffers; i++) {
-					BufferOrEvent next = seq.getNext();
-					assertNotNull(next);
-					if (next.isEvent()) {
-						BufferOrEvent expected = events.get(numEvent++);
-						assertEquals(expected.getEvent(), next.getEvent());
-						assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-					}
-					else {
-						validateBuffer(next, bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels));
-					}
-				}
-
-				// no further data
-				assertNull(seq.getNext());
-
-				// all events need to be consumed
-				assertEquals(events.size(), numEvent);
-				
-				seq.cleanup();
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSpillWhileReading() {
-		LOG.info("Starting SpillWhileReading test");
-		
-		try {
-			final int sequences = 10;
-			
-			final Random rnd = new Random();
-			
-			final int maxNumEventsAndBuffers = 30000;
-			final int maxNumChannels = 1656;
-			
-			int sequencesConsumed = 0;
-			
-			ArrayDeque<SequenceToConsume> pendingSequences = new ArrayDeque<SequenceToConsume>();
-			SequenceToConsume currentSequence = null;
-			int currentNumEvents = 0;
-			int currentNumRecordAndEvents = 0;
-			
-			// do multiple spilling / rolling over rounds
-			for (int round = 0; round < 2*sequences; round++) {
-
-				if (round % 2 == 1) {
-					// make this an empty sequence
-					assertNull(spiller.rollOver());
-				}
-				else {
-					// proper spilled sequence
-					final long bufferSeed = rnd.nextLong();
-					final Random bufferRnd = new Random(bufferSeed);
-					
-					final int numEventsAndBuffers = rnd.nextInt(maxNumEventsAndBuffers) + 1;
-					final int numChannels = rnd.nextInt(maxNumChannels) + 1;
-	
-					final ArrayList<BufferOrEvent> events = new ArrayList<BufferOrEvent>(128);
-	
-					int generated = 0;
-					while (generated < numEventsAndBuffers) {
-						
-						if (currentSequence == null || rnd.nextDouble() < 0.5) {
-							// add a new record
-							boolean isEvent = rnd.nextDouble() < 0.05;
-							if (isEvent) {
-								BufferOrEvent evt = generateRandomEvent(rnd, numChannels);
-								events.add(evt);
-								spiller.add(evt);
-							}
-							else {
-								BufferOrEvent evt = generateRandomBuffer(bufferRnd.nextInt(PAGE_SIZE) + 1, bufferRnd.nextInt(numChannels));
-								spiller.add(evt);
-							}
-							generated++;
-						}
-						else {
-							// consume a record
-							BufferOrEvent next = currentSequence.sequence.getNext();
-							assertNotNull(next);
-							if (next.isEvent()) {
-								BufferOrEvent expected = currentSequence.events.get(currentNumEvents++);
-								assertEquals(expected.getEvent(), next.getEvent());
-								assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-							}
-							else {
-								Random validationRnd = currentSequence.bufferRnd;
-								validateBuffer(next, validationRnd.nextInt(PAGE_SIZE) + 1, validationRnd.nextInt(currentSequence.numChannels));
-							}
-							
-							currentNumRecordAndEvents++;
-							if (currentNumRecordAndEvents == currentSequence.numBuffersAndEvents) {
-								// done with the sequence
-								currentSequence.sequence.cleanup();
-								sequencesConsumed++;
-								
-								// validate we had all events
-								assertEquals(currentSequence.events.size(), currentNumEvents);
-								
-								// reset
-								currentSequence = pendingSequences.pollFirst();
-								if (currentSequence != null) {
-									currentSequence.sequence.open();
-								}
-								
-								currentNumRecordAndEvents = 0;
-								currentNumEvents = 0;
-							}
-						}
-					}
-	
-					// done generating a sequence. queue it for consumption
-					bufferRnd.setSeed(bufferSeed);
-					BufferSpiller.SpilledBufferOrEventSequence seq = spiller.rollOver();
-					
-					SequenceToConsume stc = new SequenceToConsume(bufferRnd, events, seq, numEventsAndBuffers, numChannels);
-					
-					if (currentSequence == null) {
-						currentSequence = stc;
-						stc.sequence.open();
-					}
-					else {
-						pendingSequences.addLast(stc);
-					}
-				}
-			}
-			
-			// consume all the remainder
-			while (currentSequence != null) {
-				// consume a record
-				BufferOrEvent next = currentSequence.sequence.getNext();
-				assertNotNull(next);
-				if (next.isEvent()) {
-					BufferOrEvent expected = currentSequence.events.get(currentNumEvents++);
-					assertEquals(expected.getEvent(), next.getEvent());
-					assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-				}
-				else {
-					Random validationRnd = currentSequence.bufferRnd;
-					validateBuffer(next, validationRnd.nextInt(PAGE_SIZE) + 1, validationRnd.nextInt(currentSequence.numChannels));
-				}
-
-				currentNumRecordAndEvents++;
-				if (currentNumRecordAndEvents == currentSequence.numBuffersAndEvents) {
-					// done with the sequence
-					currentSequence.sequence.cleanup();
-					sequencesConsumed++;
-
-					// validate we had all events
-					assertEquals(currentSequence.events.size(), currentNumEvents);
-
-					// reset
-					currentSequence = pendingSequences.pollFirst();
-					if (currentSequence != null) {
-						currentSequence.sequence.open();
-					}
-
-					currentNumRecordAndEvents = 0;
-					currentNumEvents = 0;
-				}
-			}
-			
-			assertEquals(sequences, sequencesConsumed);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utils
-	// ------------------------------------------------------------------------
-	
-	private static BufferOrEvent generateRandomEvent(Random rnd, int numChannels) {
-		long magicNumber = rnd.nextLong();
-		byte[] data = new byte[rnd.nextInt(1000)];
-		rnd.nextBytes(data);
-		TestEvent evt = new TestEvent(magicNumber, data);
-
-		int channelIndex = rnd.nextInt(numChannels);
-		
-		return new BufferOrEvent(evt, channelIndex);
-	}
-
-	private static BufferOrEvent generateRandomBuffer(int size, int channelIndex) {
-		MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE);
-		for (int i = 0; i < size; i++) {
-			seg.put(i, (byte) i);
-		}
-		
-		Buffer buf = new Buffer(seg, FreeingBufferRecycler.INSTANCE);
-		buf.setSize(size);
-		return new BufferOrEvent(buf, channelIndex);
-	}
-
-	private static void validateBuffer(BufferOrEvent boe, int expectedSize, int expectedChannelIndex) {
-		assertEquals("wrong channel index", expectedChannelIndex, boe.getChannelIndex());
-		assertTrue("is not buffer", boe.isBuffer());
-
-		Buffer buf = boe.getBuffer();
-		assertEquals("wrong buffer size", expectedSize, buf.getSize());
-
-		MemorySegment seg = buf.getMemorySegment();
-		for (int i = 0; i < expectedSize; i++) {
-			byte expected = (byte) i;
-			if (expected != seg.get(i)) {
-				fail(String.format(
-						"wrong buffer contents at position %s : expected=%d , found=%d", i, expected, seg.get(i)));
-			}
-		}
-	}
-
-	private static void checkNoTempFilesRemain() {
-		// validate that all temp files have been removed
-		for (File dir : IO_MANAGER.getSpillingDirectories()) {
-			for (String file : dir.list()) {
-				if (file != null && !(file.equals(".") || file.equals(".."))) {
-					fail("barrier buffer did not clean up temp files. remaining file: " + file);
-				}
-			}
-		}
-	}
-	
-	private static class SequenceToConsume {
-
-		final BufferSpiller.SpilledBufferOrEventSequence sequence;
-		final ArrayList<BufferOrEvent> events;
-		final Random bufferRnd;
-		final int numBuffersAndEvents;
-		final int numChannels;
-
-		private SequenceToConsume(Random bufferRnd, ArrayList<BufferOrEvent> events,
-									BufferSpiller.SpilledBufferOrEventSequence sequence,
-									int numBuffersAndEvents, int numChannels) {
-			this.bufferRnd = bufferRnd;
-			this.events = events;
-			this.sequence = sequence;
-			this.numBuffersAndEvents = numBuffersAndEvents;
-			this.numChannels = numChannels;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
deleted file mode 100644
index cb8a058..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.io;
-
-import org.apache.flink.runtime.event.TaskEvent;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.util.event.EventListener;
-
-import java.util.ArrayDeque;
-import java.util.List;
-import java.util.Queue;
-
-public class MockInputGate implements InputGate {
-
-	private final int pageSize;
-	
-	private final int numChannels;
-	
-	private final Queue<BufferOrEvent> boes;
-
-	private final boolean[] closed;
-	
-	private int closedChannels;
-
-	
-	public MockInputGate(int pageSize, int numChannels, List<BufferOrEvent> boes) {
-		this.pageSize = pageSize;
-		this.numChannels = numChannels;
-		this.boes = new ArrayDeque<BufferOrEvent>(boes);
-		this.closed = new boolean[numChannels];
-	}
-
-	@Override
-	public int getPageSize() {
-		return pageSize;
-	}
-	
-	@Override
-	public int getNumberOfInputChannels() {
-		return numChannels;
-	}
-
-	@Override
-	public boolean isFinished() {
-		return boes.isEmpty();
-	}
-
-	@Override
-	public BufferOrEvent getNextBufferOrEvent() {
-		BufferOrEvent next = boes.poll();
-		if (next == null) {
-			return null;
-		}
-		
-		int channelIdx = next.getChannelIndex();
-		if (closed[channelIdx]) {
-			throw new RuntimeException("Inconsistent: Channel " + channelIdx
-					+ " has data even though it is already closed.");
-		}
-		if (next.isEvent() && next.getEvent() instanceof EndOfPartitionEvent) {
-			closed[channelIdx] = true;
-			closedChannels++;
-		}
-		return next;
-	}
-
-	@Override
-	public void requestPartitions() {}
-
-	@Override
-	public void sendTaskEvent(TaskEvent event) {}
-
-	@Override
-	public void registerListener(EventListener<InputGate> listener) {}
-	
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java
deleted file mode 100644
index 991b033..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/SpilledBufferOrEventSequenceTest.java
+++ /dev/null
@@ -1,482 +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.io;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.streaming.runtime.io.BufferSpiller.SpilledBufferOrEventSequence;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
-import java.util.ArrayList;
-import java.util.Random;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests that validate the behavior of the {@link SpilledBufferOrEventSequence} in isolation,
- * with respect to detecting corrupt sequences, trailing data, and interleaved buffers and events.
- */
-public class SpilledBufferOrEventSequenceTest {
-	
-	private final ByteBuffer buffer = ByteBuffer.allocateDirect(128 * 1024).order(ByteOrder.LITTLE_ENDIAN);
-	private final int pageSize = 32*1024;
-	
-	private File tempFile;
-	private FileChannel fileChannel;
-	
-	
-	@Before
-	public void initTempChannel() {
-		try {
-			tempFile = File.createTempFile("testdata", "tmp");
-			fileChannel = new RandomAccessFile(tempFile, "rw").getChannel();
-		}
-		catch (Exception e) {
-			cleanup();
-		}
-	}
-	
-	@After
-	public void cleanup() {
-		if (fileChannel != null) {
-			try {
-				fileChannel.close();
-			}
-			catch (IOException e) {
-				// ignore
-			}
-		}
-		if (tempFile != null) {
-			//noinspection ResultOfMethodCallIgnored
-			tempFile.delete();
-		}
-	}
-	
-	
-	// ------------------------------------------------------------------------
-	//  Tests
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void testEmptyChannel() {
-		try {
-			SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			seq.open();
-			
-			assertNull(seq.getNext());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testIncompleteHeaderOnFirstElement() {
-		try {
-			ByteBuffer buf = ByteBuffer.allocate(7);
-			buf.order(ByteOrder.LITTLE_ENDIAN);
-			
-			fileChannel.write(buf);
-			fileChannel.position(0);
-			
-			SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			seq.open();
-			
-			try {
-				seq.getNext();
-				fail("should fail with an exception");
-			}
-			catch (IOException e) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testBufferSequence() {
-		try {
-			final Random rnd = new Random();
-			final long seed = rnd.nextLong();
-			
-			final int numBuffers = 325;
-			final int numChannels = 671;
-			
-			rnd.setSeed(seed);
-			
-			for (int i = 0; i < numBuffers; i++) {
-				writeBuffer(fileChannel, rnd.nextInt(pageSize) + 1, rnd.nextInt(numChannels));
-			}
-
-			fileChannel.position(0L);
-			rnd.setSeed(seed);
-
-			SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			seq.open();
-			
-			for (int i = 0; i < numBuffers; i++) {
-				validateBuffer(seq.getNext(), rnd.nextInt(pageSize) + 1, rnd.nextInt(numChannels));
-			}
-			
-			// should have no more data
-			assertNull(seq.getNext());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testBufferSequenceWithIncompleteBuffer() {
-		try {
-			writeBuffer(fileChannel, 1672, 7);
-			
-			// write an incomplete buffer
-			ByteBuffer data = ByteBuffer.allocate(615);
-			data.order(ByteOrder.LITTLE_ENDIAN);
-			
-			data.putInt(2);
-			data.putInt(999);
-			data.put((byte) 0);
-			data.position(0);
-			data.limit(312);
-			fileChannel.write(data);
-			fileChannel.position(0L);
-
-			SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			seq.open();
-			
-			// first one is valid
-			validateBuffer(seq.getNext(), 1672, 7);
-			
-			// next one should fail
-			try {
-				seq.getNext();
-				fail("should fail with an exception");
-			}
-			catch (IOException e) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testEventSequence() {
-		try {
-			final Random rnd = new Random();
-			final int numEvents = 3000;
-			final int numChannels = 1656;
-			
-			final ArrayList<BufferOrEvent> events = new ArrayList<BufferOrEvent>(numEvents);
-			
-			for (int i = 0; i < numEvents; i++) {
-				events.add(generateAndWriteEvent(fileChannel, rnd, numChannels));
-			}
-
-			fileChannel.position(0L);
-			SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			seq.open();
-			
-			int i = 0;
-			BufferOrEvent boe;
-			while ((boe = seq.getNext()) != null) {
-				BufferOrEvent expected = events.get(i);
-				assertTrue(boe.isEvent());
-				assertEquals(expected.getEvent(), boe.getEvent());
-				assertEquals(expected.getChannelIndex(), boe.getChannelIndex());
-				i++;
-			}
-			
-			assertEquals(numEvents, i);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testMixedSequence() {
-		try {
-			final Random rnd = new Random();
-			final Random bufferRnd = new Random();
-
-			final long bufferSeed = rnd.nextLong();
-			bufferRnd.setSeed(bufferSeed);
-			
-			final int numEventsAndBuffers = 3000;
-			final int numChannels = 1656;
-
-			final ArrayList<BufferOrEvent> events = new ArrayList<BufferOrEvent>(128);
-
-			// generate sequence
-			
-			for (int i = 0; i < numEventsAndBuffers; i++) {
-				boolean isEvent = rnd.nextDouble() < 0.05d;
-				if (isEvent) {
-					events.add(generateAndWriteEvent(fileChannel, rnd, numChannels));
-				}
-				else {
-					writeBuffer(fileChannel, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels));
-				}
-			}
-			
-			// reset and create reader
-			
-			fileChannel.position(0L);
-			bufferRnd.setSeed(bufferSeed);
-			SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			seq.open();
-			
-			// read and validate the sequence
-			
-			int numEvent = 0;
-			for (int i = 0; i < numEventsAndBuffers; i++) {
-				BufferOrEvent next = seq.getNext();
-				if (next.isEvent()) {
-					BufferOrEvent expected = events.get(numEvent++);
-					assertEquals(expected.getEvent(), next.getEvent());
-					assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-				}
-				else {
-					validateBuffer(next, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels));
-				}
-			}
-			
-			// no further data
-			assertNull(seq.getNext());
-			
-			// all events need to be consumed
-			assertEquals(events.size(), numEvent);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testMultipleSequences() {
-		File secondFile = null;
-		FileChannel secondChannel = null;
-		
-		try {
-			// create the second file channel
-			secondFile = File.createTempFile("testdata", "tmp");
-			secondChannel = new RandomAccessFile(secondFile, "rw").getChannel();
-			
-			final Random rnd = new Random();
-			final Random bufferRnd = new Random();
-
-			final long bufferSeed = rnd.nextLong();
-			bufferRnd.setSeed(bufferSeed);
-
-			final int numEventsAndBuffers1 = 272;
-			final int numEventsAndBuffers2 = 151;
-			
-			final int numChannels = 1656;
-
-			final ArrayList<BufferOrEvent> events1 = new ArrayList<BufferOrEvent>(128);
-			final ArrayList<BufferOrEvent> events2 = new ArrayList<BufferOrEvent>(128);
-
-			// generate sequence 1
-
-			for (int i = 0; i < numEventsAndBuffers1; i++) {
-				boolean isEvent = rnd.nextDouble() < 0.05d;
-				if (isEvent) {
-					events1.add(generateAndWriteEvent(fileChannel, rnd, numChannels));
-				}
-				else {
-					writeBuffer(fileChannel, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels));
-				}
-			}
-
-			// generate sequence 2
-
-			for (int i = 0; i < numEventsAndBuffers2; i++) {
-				boolean isEvent = rnd.nextDouble() < 0.05d;
-				if (isEvent) {
-					events2.add(generateAndWriteEvent(secondChannel, rnd, numChannels));
-				}
-				else {
-					writeBuffer(secondChannel, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels));
-				}
-			}
-
-			// reset and create reader
-
-			fileChannel.position(0L);
-			secondChannel.position(0L);
-			
-			bufferRnd.setSeed(bufferSeed);
-			
-			SpilledBufferOrEventSequence seq1 = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			SpilledBufferOrEventSequence seq2 = new SpilledBufferOrEventSequence(secondFile, secondChannel, buffer, pageSize);
-
-			// read and validate the sequence 1
-			seq1.open();
-
-			int numEvent = 0;
-			for (int i = 0; i < numEventsAndBuffers1; i++) {
-				BufferOrEvent next = seq1.getNext();
-				if (next.isEvent()) {
-					BufferOrEvent expected = events1.get(numEvent++);
-					assertEquals(expected.getEvent(), next.getEvent());
-					assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-				}
-				else {
-					validateBuffer(next, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels));
-				}
-			}
-			assertNull(seq1.getNext());
-			assertEquals(events1.size(), numEvent);
-
-			// read and validate the sequence 2
-			seq2.open();
-
-			numEvent = 0;
-			for (int i = 0; i < numEventsAndBuffers2; i++) {
-				BufferOrEvent next = seq2.getNext();
-				if (next.isEvent()) {
-					BufferOrEvent expected = events2.get(numEvent++);
-					assertEquals(expected.getEvent(), next.getEvent());
-					assertEquals(expected.getChannelIndex(), next.getChannelIndex());
-				}
-				else {
-					validateBuffer(next, bufferRnd.nextInt(pageSize) + 1, bufferRnd.nextInt(numChannels));
-				}
-			}
-			assertNull(seq2.getNext());
-			assertEquals(events2.size(), numEvent);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			if (secondChannel != null) {
-				try {
-					secondChannel.close();
-				}
-				catch (IOException e) {
-					// ignore here
-				}
-			}
-			if (secondFile != null) {
-				//noinspection ResultOfMethodCallIgnored
-				secondFile.delete();
-			}
-		}
-	}
-
-	@Test
-	public void testCleanup() {
-		try {
-			ByteBuffer data = ByteBuffer.allocate(157);
-			data.order(ByteOrder.LITTLE_ENDIAN);
-			
-			fileChannel.write(data);
-			fileChannel.position(54);
-			
-			SpilledBufferOrEventSequence seq = new SpilledBufferOrEventSequence(tempFile, fileChannel, buffer, pageSize);
-			seq.open();
-			seq.cleanup();
-			
-			assertFalse(fileChannel.isOpen());
-			assertFalse(tempFile.exists());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utils
-	// ------------------------------------------------------------------------
-
-	private static BufferOrEvent generateAndWriteEvent(FileChannel fileChannel, Random rnd, int numChannels) throws IOException {
-		long magicNumber = rnd.nextLong();
-		byte[] data = new byte[rnd.nextInt(1000)];
-		rnd.nextBytes(data);
-		TestEvent evt = new TestEvent(magicNumber, data);
-		
-		int channelIndex = rnd.nextInt(numChannels);
-		
-		ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(evt);
-		ByteBuffer header = ByteBuffer.allocate(9);
-		header.order(ByteOrder.LITTLE_ENDIAN);
-		
-		header.putInt(channelIndex);
-		header.putInt(serializedEvent.remaining());
-		header.put((byte) 1);
-		header.flip();
-		
-		fileChannel.write(header);
-		fileChannel.write(serializedEvent);
-		return new BufferOrEvent(evt, channelIndex);
-	}
-	
-	private static void writeBuffer(FileChannel fileChannel, int size, int channelIndex) throws IOException {
-		ByteBuffer data = ByteBuffer.allocate(size + 9);
-		data.order(ByteOrder.LITTLE_ENDIAN);
-		
-		data.putInt(channelIndex);
-		data.putInt(size);
-		data.put((byte) 0);
-		for (int i = 0; i < size; i++) {
-			data.put((byte) i);
-		}
-		data.flip();
-		fileChannel.write(data);
-	}
-
-	private static void validateBuffer(BufferOrEvent boe, int expectedSize, int expectedChannelIndex) {
-		assertEquals("wrong channel index", expectedChannelIndex, boe.getChannelIndex());
-		assertTrue("is not buffer", boe.isBuffer());
-		
-		Buffer buf = boe.getBuffer();
-		assertEquals("wrong buffer size", expectedSize, buf.getSize());
-		
-		MemorySegment seg = buf.getMemorySegment();
-		for (int i = 0; i < expectedSize; i++) {
-			assertEquals("wrong buffer contents", (byte) i, seg.get(i));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
deleted file mode 100644
index 45bbbda..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
+++ /dev/null
@@ -1,131 +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.io;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferProvider;
-import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
-import org.apache.flink.types.LongValue;
-
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.io.IOException;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * This test uses the PowerMockRunner runner to work around the fact that the 
- * {@link ResultPartitionWriter} class is final.
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(ResultPartitionWriter.class)
-public class StreamRecordWriterTest {
-
-	/**
-	 * Verifies that exceptions during flush from the output flush thread are
-	 * recognized in the writer.
-	 */
-	@Test
-	public void testPropagateAsyncFlushError() {
-		FailingWriter<LongValue> testWriter = null;
-		try {
-			ResultPartitionWriter mockResultPartitionWriter = getMockWriter(5);
-			
-			// test writer that flushes every 5ms and fails after 3 flushes
-			testWriter = new FailingWriter<LongValue>(mockResultPartitionWriter,
-					new RoundRobinChannelSelector<LongValue>(), 5, 3);
-			
-			try {
-				long deadline = System.currentTimeMillis() + 20000; // in max 20 seconds (conservative)
-				long l = 0L;
-				
-				while (System.currentTimeMillis() < deadline) {
-					testWriter.emit(new LongValue(l++));
-				}
-				
-				fail("This should have failed with an exception");
-			}
-			catch (IOException e) {
-				assertNotNull(e.getCause());
-				assertTrue(e.getCause().getMessage().contains("Test Exception"));
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			if (testWriter != null) {
-				testWriter.close();
-			}
-		}
-	}
-	
-	private static ResultPartitionWriter getMockWriter(int numPartitions) throws Exception {
-		BufferProvider mockProvider = mock(BufferProvider.class);
-		when(mockProvider.requestBufferBlocking()).thenAnswer(new Answer<Buffer>() {
-			@Override
-			public Buffer answer(InvocationOnMock invocation) {
-				return new Buffer(
-						MemorySegmentFactory.allocateUnpooledSegment(4096),
-						FreeingBufferRecycler.INSTANCE);
-			}
-		});
-		
-		ResultPartitionWriter mockWriter = mock(ResultPartitionWriter.class);
-		when(mockWriter.getBufferProvider()).thenReturn(mockProvider);
-		when(mockWriter.getNumberOfOutputChannels()).thenReturn(numPartitions);
-		
-		
-		return mockWriter;
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private static class FailingWriter<T extends IOReadableWritable> extends StreamRecordWriter<T> {
-		
-		private int flushesBeforeException;
-		
-		private FailingWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSelector,
-								long timeout, int flushesBeforeException) {
-			super(writer, channelSelector, timeout);
-			this.flushesBeforeException = flushesBeforeException;
-		}
-
-		@Override
-		public void flush() throws IOException {
-			if (flushesBeforeException-- <= 0) {
-				throw new IOException("Test Exception");
-			}
-			super.flush();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java
deleted file mode 100644
index 286477a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/TestEvent.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.io;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.util.StringUtils;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-/**
- * A simple task event, used for validation of buffer or event blocking/buffering.
- */
-public class TestEvent extends AbstractEvent {
-
-	private long magicNumber;
-
-	private byte[] payload;
-
-	public TestEvent() {}
-
-	public TestEvent(long magicNumber, byte[] payload) {
-		this.magicNumber = magicNumber;
-		this.payload = payload;
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  Serialization
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		out.writeLong(magicNumber);
-		out.writeInt(payload.length);
-		out.write(payload);
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		this.magicNumber = in.readLong();
-		this.payload = new byte[in.readInt()];
-		in.read(this.payload);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Standard utilities
-	// ------------------------------------------------------------------------
-
-	@Override
-	public int hashCode() {
-		return Long.valueOf(magicNumber).hashCode();
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj != null && obj.getClass() == TestEvent.class) {
-			TestEvent that = (TestEvent) obj;
-			return this.magicNumber == that.magicNumber && Arrays.equals(this.payload, that.payload);
-		}
-		else {
-			return false;
-		}
-	}
-
-	@Override
-	public String toString() {
-		return String.format("TestEvent %d (%s)", magicNumber, StringUtils.byteToHexString(payload));
-	}
-}
\ No newline at end of file


[33/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java
deleted file mode 100644
index 2e415f4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/windows/Window.java
+++ /dev/null
@@ -1,31 +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.windows;
-
-/**
- * A {@code Window} is a grouping of elements into finite buckets. Windows have a maximum timestamp
- * which means that, at some point, all elements that go into one window will have arrived.
- *
- * <p>
- * Subclasses should implement {@code equals()} and {@code hashCode()} so that logically
- * same windows are treated the same.
- */
-public abstract class Window {
-
-	public abstract long maxTimestamp();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
deleted file mode 100644
index 863f7ac..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierBuffer.java
+++ /dev/null
@@ -1,320 +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.io;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The barrier buffer is {@link CheckpointBarrierHandler} that blocks inputs with barriers until
- * all inputs have received the barrier for a given checkpoint.
- * 
- * <p>To avoid back-pressuring the input streams (which may cause distributed deadlocks), the
- * BarrierBuffer continues receiving buffers from the blocked channels and stores them internally until 
- * the blocks are released.</p>
- */
-public class BarrierBuffer implements CheckpointBarrierHandler {
-
-	private static final Logger LOG = LoggerFactory.getLogger(BarrierBuffer.class);
-	
-	/** The gate that the buffer draws its input from */
-	private final InputGate inputGate;
-
-	/** Flags that indicate whether a channel is currently blocked/buffered */
-	private final boolean[] blockedChannels;
-	
-	/** The total number of channels that this buffer handles data from */
-	private final int totalNumberOfInputChannels;
-	
-	/** To utility to write blocked data to a file channel */
-	private final BufferSpiller bufferSpiller;
-
-	/** The pending blocked buffer/event sequences. Must be consumed before requesting
-	 * further data from the input gate. */
-	private final ArrayDeque<BufferSpiller.SpilledBufferOrEventSequence> queuedBuffered;
-
-	/** The sequence of buffers/events that has been unblocked and must now be consumed
-	 * before requesting further data from the input gate */
-	private BufferSpiller.SpilledBufferOrEventSequence currentBuffered;
-
-	/** Handler that receives the checkpoint notifications */
-	private EventListener<CheckpointBarrier> checkpointHandler;
-
-	/** The ID of the checkpoint for which we expect barriers */
-	private long currentCheckpointId = -1L;
-
-	/** The number of received barriers (= number of blocked/buffered channels) */
-	private int numBarriersReceived;
-	
-	/** The number of already closed channels */
-	private int numClosedChannels;
-	
-	/** Flag to indicate whether we have drawn all available input */
-	private boolean endOfStream;
-
-	/**
-	 * 
-	 * @param inputGate The input gate to draw the buffers and events from.
-	 * @param ioManager The I/O manager that gives access to the temp directories.
-	 * 
-	 * @throws IOException Thrown, when the spilling to temp files cannot be initialized.
-	 */
-	public BarrierBuffer(InputGate inputGate, IOManager ioManager) throws IOException {
-		this.inputGate = inputGate;
-		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
-		this.blockedChannels = new boolean[this.totalNumberOfInputChannels];
-		
-		this.bufferSpiller = new BufferSpiller(ioManager, inputGate.getPageSize());
-		this.queuedBuffered = new ArrayDeque<BufferSpiller.SpilledBufferOrEventSequence>();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Buffer and barrier handling
-	// ------------------------------------------------------------------------
-
-	@Override
-	public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException {
-		while (true) {
-			// process buffered BufferOrEvents before grabbing new ones
-			BufferOrEvent next;
-			if (currentBuffered == null) {
-				next = inputGate.getNextBufferOrEvent();
-			}
-			else {
-				next = currentBuffered.getNext();
-				if (next == null) {
-					completeBufferedSequence();
-					return getNextNonBlocked();
-				}
-			}
-			
-			if (next != null) {
-				if (isBlocked(next.getChannelIndex())) {
-					// if the channel is blocked we, we just store the BufferOrEvent
-					bufferSpiller.add(next);
-				}
-				else if (next.isBuffer()) {
-					return next;
-				}
-				else if (next.getEvent().getClass() == CheckpointBarrier.class) {
-					if (!endOfStream) {
-						// process barriers only if there is a chance of the checkpoint completing
-						processBarrier((CheckpointBarrier) next.getEvent(), next.getChannelIndex());
-					}
-				}
-				else {
-					if (next.getEvent().getClass() == EndOfPartitionEvent.class) {
-						numClosedChannels++;
-						// no chance to complete this checkpoint
-						releaseBlocks();
-					}
-					return next;
-				}
-			}
-			else if (!endOfStream) {
-				// end of stream. we feed the data that is still buffered
-				endOfStream = true;
-				releaseBlocks();
-				return getNextNonBlocked();
-			}
-			else {
-				return null;
-			}
-		}
-	}
-	
-	private void completeBufferedSequence() throws IOException {
-		currentBuffered.cleanup();
-		currentBuffered = queuedBuffered.pollFirst();
-		if (currentBuffered != null) {
-			currentBuffered.open();
-		}
-	}
-	
-	private void processBarrier(CheckpointBarrier receivedBarrier, int channelIndex) throws IOException {
-		final long barrierId = receivedBarrier.getId();
-
-		if (numBarriersReceived > 0) {
-			// subsequent barrier of a checkpoint.
-			if (barrierId == currentCheckpointId) {
-				// regular case
-				onBarrier(channelIndex);
-			}
-			else if (barrierId > currentCheckpointId) {
-				// we did not complete the current checkpoint
-				LOG.warn("Received checkpoint barrier for checkpoint {} before completing current checkpoint {}. " +
-						"Skipping current checkpoint.", barrierId, currentCheckpointId);
-
-				releaseBlocks();
-				currentCheckpointId = barrierId;
-				onBarrier(channelIndex);
-			}
-			else {
-				// ignore trailing barrier from aborted checkpoint
-				return;
-			}
-			
-		}
-		else if (barrierId > currentCheckpointId) {
-			// first barrier of a new checkpoint
-			currentCheckpointId = barrierId;
-			onBarrier(channelIndex);
-		}
-		else {
-			// trailing barrier from previous (skipped) checkpoint
-			return;
-		}
-
-		// check if we have all barriers
-		if (numBarriersReceived + numClosedChannels == totalNumberOfInputChannels) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Received all barrier, triggering checkpoint {} at {}",
-						receivedBarrier.getId(), receivedBarrier.getTimestamp());
-			}
-
-			if (checkpointHandler != null) {
-				checkpointHandler.onEvent(receivedBarrier);
-			}
-			
-			releaseBlocks();
-		}
-	}
-	
-	@Override
-	public void registerCheckpointEventHandler(EventListener<CheckpointBarrier> checkpointHandler) {
-		if (this.checkpointHandler == null) {
-			this.checkpointHandler = checkpointHandler;
-		}
-		else {
-			throw new IllegalStateException("BarrierBuffer already has a registered checkpoint handler");
-		}
-	}
-	
-	@Override
-	public boolean isEmpty() {
-		return currentBuffered == null;
-	}
-
-	@Override
-	public void cleanup() throws IOException {
-		bufferSpiller.close();
-		if (currentBuffered != null) {
-			currentBuffered.cleanup();
-		}
-		for (BufferSpiller.SpilledBufferOrEventSequence seq : queuedBuffered) {
-			seq.cleanup();
-		}
-	}
-	
-	/**
-	 * Checks whether the channel with the given index is blocked.
-	 * 
-	 * @param channelIndex The channel index to check.
-	 * @return True if the channel is blocked, false if not.
-	 */
-	private boolean isBlocked(int channelIndex) {
-		return blockedChannels[channelIndex];
-	}
-	
-	/**
-	 * Blocks the given channel index, from which a barrier has been received.
-	 * 
-	 * @param channelIndex The channel index to block.
-	 */
-	private void onBarrier(int channelIndex) throws IOException {
-		if (!blockedChannels[channelIndex]) {
-			blockedChannels[channelIndex] = true;
-			numBarriersReceived++;
-			
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Received barrier from channel " + channelIndex);
-			}
-		}
-		else {
-			throw new IOException("Stream corrupt: Repeated barrier for same checkpoint and input stream");
-		}
-	}
-
-	/**
-	 * Releases the blocks on all channels. Makes sure the just written data
-	 * is the next to be consumed.
-	 */
-	private void releaseBlocks() throws IOException {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Releasing blocks");
-		}
-
-		for (int i = 0; i < blockedChannels.length; i++) {
-			blockedChannels[i] = false;
-		}
-		numBarriersReceived = 0;
-
-		if (currentBuffered == null) {
-			// common case: no more buffered data
-			currentBuffered = bufferSpiller.rollOver();
-			if (currentBuffered != null) {
-				currentBuffered.open();
-			}
-		}
-		else {
-			// uncommon case: buffered data pending
-			// push back the pending data, if we have any
-			
-			// since we did not fully drain the previous sequence, we need to allocate a new buffer for this one
-			BufferSpiller.SpilledBufferOrEventSequence bufferedNow = bufferSpiller.rollOverWithNewBuffer();
-			if (bufferedNow != null) {
-				bufferedNow.open();
-				queuedBuffered.addFirst(currentBuffered);
-				currentBuffered = bufferedNow;
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	// For Testing
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the ID defining the current pending, or just completed, checkpoint.
-	 * 
-	 * @return The ID of the pending of completed checkpoint. 
-	 */
-	public long getCurrentCheckpointId() {
-		return this.currentCheckpointId;
-	}
-	
-	// ------------------------------------------------------------------------
-	// Utilities 
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return String.format("last checkpoint: %d, current barriers: %d, closed channels: %d",
-				currentCheckpointId, numBarriersReceived, numClosedChannels);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
deleted file mode 100644
index 119fb23..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BarrierTracker.java
+++ /dev/null
@@ -1,205 +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.io;
-
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-
-/**
- * The BarrierTracker keeps track of what checkpoint barriers have been received from
- * which input channels. Once it has observed all checkpoint barriers for a checkpoint ID,
- * it notifies its listener of a completed checkpoint.
- * 
- * <p>Unlike the {@link BarrierBuffer}, the BarrierTracker does not block the input
- * channels that have sent barriers, so it cannot be used to gain "exactly-once" processing
- * guarantees. It can, however, be used to gain "at least once" processing guarantees.</p>
- * 
- * <p>NOTE: This implementation strictly assumes that newer checkpoints have higher checkpoint IDs.</p>
- */
-public class BarrierTracker implements CheckpointBarrierHandler {
-
-	/** The tracker tracks a maximum number of checkpoints, for which some, but not all
-	 * barriers have yet arrived. */
-	private static final int MAX_CHECKPOINTS_TO_TRACK = 50;
-	
-	/** The input gate, to draw the buffers and events from */
-	private final InputGate inputGate;
-	
-	/** The number of channels. Once that many barriers have been received for a checkpoint,
-	 * the checkpoint is considered complete. */
-	private final int totalNumberOfInputChannels;
-
-	/** All checkpoints for which some (but not all) barriers have been received,
-	 * and that are not yet known to be subsumed by newer checkpoints */
-	private final ArrayDeque<CheckpointBarrierCount> pendingCheckpoints;
-	
-	/** The listener to be notified on complete checkpoints */
-	private EventListener<CheckpointBarrier> checkpointHandler;
-	
-	/** The highest checkpoint ID encountered so far */
-	private long latestPendingCheckpointID = -1;
-	
-	
-	public BarrierTracker(InputGate inputGate) {
-		this.inputGate = inputGate;
-		this.totalNumberOfInputChannels = inputGate.getNumberOfInputChannels();
-		this.pendingCheckpoints = new ArrayDeque<CheckpointBarrierCount>();
-	}
-
-	@Override
-	public BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException {
-		while (true) {
-			BufferOrEvent next = inputGate.getNextBufferOrEvent();
-			if (next == null) {
-				return null;
-			}
-			else if (next.isBuffer() || next.getEvent().getClass() != CheckpointBarrier.class) {
-				return next;
-			}
-			else {
-				processBarrier((CheckpointBarrier) next.getEvent());
-			}
-		}
-	}
-
-	@Override
-	public void registerCheckpointEventHandler(EventListener<CheckpointBarrier> checkpointHandler) {
-		if (this.checkpointHandler == null) {
-			this.checkpointHandler = checkpointHandler;
-		}
-		else {
-			throw new IllegalStateException("BarrierTracker already has a registered checkpoint handler");
-		}
-	}
-
-	@Override
-	public void cleanup() {
-		pendingCheckpoints.clear();
-	}
-
-	@Override
-	public boolean isEmpty() {
-		return pendingCheckpoints.isEmpty();
-	}
-
-	private void processBarrier(CheckpointBarrier receivedBarrier) {
-		// fast path for single channel trackers
-		if (totalNumberOfInputChannels == 1) {
-			if (checkpointHandler != null) {
-				checkpointHandler.onEvent(receivedBarrier);
-			}
-			return;
-		}
-		
-		// general path for multiple input channels
-		final long barrierId = receivedBarrier.getId();
-
-		// find the checkpoint barrier in the queue of bending barriers
-		CheckpointBarrierCount cbc = null;
-		int pos = 0;
-		
-		for (CheckpointBarrierCount next : pendingCheckpoints) {
-			if (next.checkpointId == barrierId) {
-				cbc = next;
-				break;
-			}
-			pos++;
-		}
-		
-		if (cbc != null) {
-			// add one to the count to that barrier and check for completion
-			int numBarriersNew = cbc.incrementBarrierCount();
-			if (numBarriersNew == totalNumberOfInputChannels) {
-				// checkpoint can be triggered
-				// first, remove this checkpoint and all all prior pending
-				// checkpoints (which are now subsumed)
-				for (int i = 0; i <= pos; i++) {
-					pendingCheckpoints.pollFirst();
-				}
-				
-				// notify the listener
-				if (checkpointHandler != null) {
-					checkpointHandler.onEvent(receivedBarrier);
-				}
-			}
-		}
-		else {
-			// first barrier for that checkpoint ID
-			// add it only if it is newer than the latest checkpoint.
-			// if it is not newer than the latest checkpoint ID, then there cannot be a
-			// successful checkpoint for that ID anyways
-			if (barrierId > latestPendingCheckpointID) {
-				latestPendingCheckpointID = barrierId;
-				pendingCheckpoints.addLast(new CheckpointBarrierCount(barrierId));
-				
-				// make sure we do not track too many checkpoints
-				if (pendingCheckpoints.size() > MAX_CHECKPOINTS_TO_TRACK) {
-					pendingCheckpoints.pollFirst();
-				}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Simple class for a checkpoint ID with a barrier counter.
-	 */
-	private static final class CheckpointBarrierCount {
-		
-		private final long checkpointId;
-		
-		private int barrierCount;
-		
-		private CheckpointBarrierCount(long checkpointId) {
-			this.checkpointId = checkpointId;
-			this.barrierCount = 1;
-		}
-
-		public int incrementBarrierCount() {
-			return ++barrierCount;
-		}
-		
-		@Override
-		public int hashCode() {
-			return (int) ((checkpointId >>> 32) ^ checkpointId) + 17 * barrierCount; 
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj instanceof  CheckpointBarrierCount) {
-				CheckpointBarrierCount that = (CheckpointBarrierCount) obj;
-				return this.checkpointId == that.checkpointId && this.barrierCount == that.barrierCount;
-			}
-			else {
-				return false;
-			}
-		}
-
-		@Override
-		public String toString() {
-			return String.format("checkpointID=%d, count=%d", checkpointId, barrierCount);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java
deleted file mode 100644
index be3c9af..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BlockingQueueBroker.java
+++ /dev/null
@@ -1,31 +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.io;
-
-import java.util.concurrent.BlockingQueue;
-
-import org.apache.flink.runtime.iterative.concurrent.Broker;
-
-public class BlockingQueueBroker extends Broker<BlockingQueue<?>> {
-	
-	/** Singleton instance */
-	public static final BlockingQueueBroker INSTANCE = new BlockingQueueBroker();
-
-	/** Cannot instantiate */
-	private BlockingQueueBroker() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
deleted file mode 100644
index cabed14..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
+++ /dev/null
@@ -1,410 +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.io;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.util.StringUtils;
-
-/**
- * The buffer spiller takes the buffers and events from a data stream and adds them to a spill file.
- * After a number of elements have been spilled, the spiller can "roll over": It presents the spilled
- * elements as a readable sequence, and opens a new spill file.
- * 
- * <p>This implementation buffers data effectively in the OS cache, which gracefully extends to the
- * disk. Most data is written and re-read milliseconds later. The file is deleted after the read.
- * Consequently, in most cases, the data will never actually hit the physical disks.</p>
- * 
- * <p>IMPORTANT: The SpilledBufferOrEventSequences created by this spiller all reuse the same
- * reading memory (to reduce overhead) and can consequently not be read concurrently.</p>
- */
-public class BufferSpiller {
-
-	/** The counter that selects the next directory to spill into */
-	private static final AtomicInteger DIRECTORY_INDEX = new AtomicInteger(0);
-	
-	/** The size of the buffer with which data is read back in */
-	private static final int READ_BUFFER_SIZE = 1024 * 1024;
-	
-	/** The directories to spill to */
-	private final File tempDir;
-	
-	/** The name prefix for spill files */
-	private final String spillFilePrefix;
-	
-	/** The buffer used for bulk reading data (used in the SpilledBufferOrEventSequence) */
-	private final ByteBuffer readBuffer;
-	
-	/** The buffer that encodes the spilled header */
-	private final ByteBuffer headBuffer;
-	
-	/** The reusable array that holds header and contents buffers */
-	private final ByteBuffer[] sources;
-	
-	/** The file that we currently spill to */
-	private File currentSpillFile;
-	
-	/** The channel of the file we currently spill to */
-	private FileChannel currentChannel;
-
-	/** The page size, to let this reader instantiate properly sized memory segments */
-	private final int pageSize;
-	
-	/** A counter, to created numbered spill files */
-	private int fileCounter;
-	
-	/** A flag to check whether the spiller has written since the last roll over */
-	private boolean hasWritten;
-	
-	/**
-	 * Creates a new buffer spiller, spilling to one of the I/O manager's temp directories.
-	 * 
-	 * @param ioManager The I/O manager for access to teh temp directories.
-	 * @param pageSize The page size used to re-create spilled buffers.
-	 * @throws IOException Thrown if the temp files for spilling cannot be initialized.
-	 */
-	public BufferSpiller(IOManager ioManager, int pageSize) throws IOException {
-		this.pageSize = pageSize;
-		
-		this.readBuffer = ByteBuffer.allocateDirect(READ_BUFFER_SIZE);
-		this.readBuffer.order(ByteOrder.LITTLE_ENDIAN);
-		
-		this.headBuffer = ByteBuffer.allocateDirect(16);
-		this.headBuffer.order(ByteOrder.LITTLE_ENDIAN);
-		
-		this.sources = new ByteBuffer[] { this.headBuffer, null };
-		
-		File[] tempDirs = ioManager.getSpillingDirectories();
-		this.tempDir = tempDirs[DIRECTORY_INDEX.getAndIncrement() % tempDirs.length];
-		
-		byte[] rndBytes = new byte[32];
-		new Random().nextBytes(rndBytes);
-		this.spillFilePrefix = StringUtils.byteToHexString(rndBytes) + '.';
-		
-		// prepare for first contents
-		createSpillingChannel();
-	}
-
-	/**
-	 * Adds a buffer or event to the sequence of spilled buffers and events.
-	 * 
-	 * @param boe The buffer or event to add and spill.
-	 * @throws IOException Thrown, if the buffer of event could not be spilled.
-	 */
-	public void add(BufferOrEvent boe) throws IOException {
-		hasWritten = true;
-		try {
-			ByteBuffer contents;
-			if (boe.isBuffer()) {
-				Buffer buf = boe.getBuffer();
-				contents = buf.getMemorySegment().wrap(0, buf.getSize());
-			}
-			else {
-				contents = EventSerializer.toSerializedEvent(boe.getEvent());
-			}
-			
-			headBuffer.clear();
-			headBuffer.putInt(boe.getChannelIndex());
-			headBuffer.putInt(contents.remaining());
-			headBuffer.put((byte) (boe.isBuffer() ? 0 : 1));
-			headBuffer.flip();
-			
-			sources[1] = contents;
-			currentChannel.write(sources);
-		}
-		finally {
-			if (boe.isBuffer()) {
-				boe.getBuffer().recycle();
-			}
-		}
-	}
-
-	/**
-	 * Starts a new sequence of spilled buffers and event and returns the current sequence of spilled buffers
-	 * for reading. This method returns {@code null}, if nothing was added since the creation of the spiller, or the
-	 * last call to this method.
-	 * 
-	 * <p>NOTE: The SpilledBufferOrEventSequences created by this method all reuse the same
-	 * reading memory (to reduce overhead) and can consequently not be read concurrently with each other.
-	 * To create a sequence that can be read concurrently with the previous SpilledBufferOrEventSequence, use the
-	 * {@link #rollOverWithNewBuffer()} method.</p>
-	 * 
-	 * @return The readable sequence of spilled buffers and events, or 'null', if nothing was added.
-	 * @throws IOException Thrown, if the readable sequence could not be created, or no new spill
-	 *                     file could be created.
-	 */
-	public SpilledBufferOrEventSequence rollOver() throws IOException {
-		return rollOverInternal(false);
-	}
-
-	/**
-	 * Starts a new sequence of spilled buffers and event and returns the current sequence of spilled buffers
-	 * for reading. This method returns {@code null}, if nothing was added since the creation of the spiller, or the
-	 * last call to this method.
-	 * 
-	 * <p>The SpilledBufferOrEventSequence returned by this method is safe for concurrent consumption with
-	 * any previously returned sequence.</p>
-	 *
-	 * @return The readable sequence of spilled buffers and events, or 'null', if nothing was added.
-	 * @throws IOException Thrown, if the readable sequence could not be created, or no new spill
-	 *                     file could be created.
-	 */
-	public SpilledBufferOrEventSequence rollOverWithNewBuffer() throws IOException {
-		return rollOverInternal(true);
-	}
-	
-	private SpilledBufferOrEventSequence rollOverInternal(boolean newBuffer) throws IOException {
-		if (!hasWritten) {
-			return null;
-		}
-		
-		ByteBuffer buf;
-		if (newBuffer) {
-			buf = ByteBuffer.allocateDirect(READ_BUFFER_SIZE);
-			buf.order(ByteOrder.LITTLE_ENDIAN);
-		} else {
-			buf = readBuffer;
-		}
-		
-		// create a reader for the spilled data
-		currentChannel.position(0L);
-		SpilledBufferOrEventSequence seq = 
-				new SpilledBufferOrEventSequence(currentSpillFile, currentChannel, buf, pageSize);
-		
-		// create ourselves a new spill file
-		createSpillingChannel();
-		
-		hasWritten = false;
-		return seq;
-	}
-
-	/**
-	 * Cleans up the current spilling channel and file.
-	 * 
-	 * Does not clean up the SpilledBufferOrEventSequences generated by calls to 
-	 * {@link #rollOver()}.
-	 * 
-	 * @throws IOException Thrown if channel closing or file deletion fail.
-	 */
-	public void close() throws IOException {
-		currentChannel.close();
-		if (!currentSpillFile.delete()) {
-			throw new IOException("Cannot delete spill file");
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  For testing
-	// ------------------------------------------------------------------------
-
-	File getCurrentSpillFile() {
-		return currentSpillFile;
-	}
-	
-	FileChannel getCurrentChannel() {
-		return currentChannel;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-	
-	@SuppressWarnings("resource")
-	private void createSpillingChannel() throws IOException {
-		currentSpillFile = new File(tempDir, spillFilePrefix + (fileCounter++) +".buffer");
-		currentChannel = new RandomAccessFile(currentSpillFile, "rw").getChannel();
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * This class represents a sequence of spilled buffers and events, created by the
-	 * {@link BufferSpiller}. The sequence of buffers and events can be read back using the
-	 * method {@link #getNext()}.
-	 */
-	public static class SpilledBufferOrEventSequence {
-		
-		/** Header is "channel index" (4 bytes) + length (4 bytes) + buffer/event (1 byte) */
-		private static final int HEADER_LENGTH = 9;
-
-		/** The file containing the data */
-		private final File file;
-		
-		/** The file channel to draw the data from */
-		private final FileChannel fileChannel;
-		
-		/** The byte buffer for bulk reading */
-		private final ByteBuffer buffer;
-
-		/** The page size to instantiate properly sized memory segments */
-		private final int pageSize;
-
-		/** Flag to track whether the sequence has been opened already */
-		private boolean opened = false;
-
-		/**
-		 * Create a reader that reads a sequence of spilled buffers and events.
-		 * 
-		 * @param file The file with the data.
-		 * @param fileChannel The file channel to read the data from.
-		 * @param buffer The buffer used for bulk reading.
-		 * @param pageSize The page size to use for the created memory segments.
-		 */
-		SpilledBufferOrEventSequence(File file, FileChannel fileChannel, ByteBuffer buffer, int pageSize) {
-			this.file = file;
-			this.fileChannel = fileChannel;
-			this.buffer = buffer;
-			this.pageSize = pageSize;
-		}
-
-		/**
-		 * Initializes the sequence for reading.
-		 * This method needs to be called before the first call to {@link #getNext()}. Otherwise
-		 * the results of {@link #getNext()} are not predictable.
-		 */
-		public void open() {
-			if (!opened) {
-				opened = true;
-				buffer.position(0);
-				buffer.limit(0);
-			}
-		}
-
-		/**
-		 * Gets the next BufferOrEvent from the spilled sequence, or {@code null}, if the
-		 * sequence is exhausted.
-		 *         
-		 * @return The next BufferOrEvent from the spilled sequence, or {@code null} (end of sequence).
-		 * @throws IOException Thrown, if the reads failed, of if the byte stream is corrupt.
-		 */
-		public BufferOrEvent getNext() throws IOException {
-			if (buffer.remaining() < HEADER_LENGTH) {
-				buffer.compact();
-				
-				while (buffer.position() < HEADER_LENGTH) {
-					if (fileChannel.read(buffer) == -1) {
-						if (buffer.position() == 0) {
-							// no trailing data
-							return null;
-						} else {
-							throw new IOException("Found trailing incomplete buffer or event");
-						}
-					}
-				}
-				
-				buffer.flip();
-			}
-			
-			final int channel = buffer.getInt();
-			final int length = buffer.getInt();
-			final boolean isBuffer = buffer.get() == 0;
-			
-			
-			if (isBuffer) {
-				// deserialize buffer
-				if (length > pageSize) {
-					throw new IOException(String.format(
-							"Spilled buffer (%d bytes) is larger than page size of (%d bytes)", length, pageSize));
-				}
-
-				MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(pageSize);
-				
-				int segPos = 0;
-				int bytesRemaining = length;
-				
-				while (true) {
-					int toCopy = Math.min(buffer.remaining(), bytesRemaining);
-					if (toCopy > 0) {
-						seg.put(segPos, buffer, toCopy);
-						segPos += toCopy;
-						bytesRemaining -= toCopy;
-					}
-					
-					if (bytesRemaining == 0) {
-						break;
-					}
-					else {
-						buffer.clear();
-						if (fileChannel.read(buffer) == -1) {
-							throw new IOException("Found trailing incomplete buffer");
-						}
-						buffer.flip();
-					}
-				}
-				
-				
-				Buffer buf = new Buffer(seg, FreeingBufferRecycler.INSTANCE);
-				buf.setSize(length);
-				
-				return new BufferOrEvent(buf, channel);
-			}
-			else {
-				// deserialize event
-				if (length > buffer.capacity() - HEADER_LENGTH) {
-					throw new IOException("Event is too large");
-				}
-
-				if (buffer.remaining() < length) {
-					buffer.compact();
-
-					while (buffer.position() < length) {
-						if (fileChannel.read(buffer) == -1) {
-							throw new IOException("Found trailing incomplete event");
-						}
-					}
-
-					buffer.flip();
-				}
-
-				int oldLimit = buffer.limit();
-				buffer.limit(buffer.position() + length);
-				AbstractEvent evt = EventSerializer.fromSerializedEvent(buffer, getClass().getClassLoader());
-				buffer.limit(oldLimit);
-				
-				return new BufferOrEvent(evt, channel);
-			}
-		}
-
-		/**
-		 * Cleans up all file resources held by this spilled sequence.
-		 * 
-		 * @throws IOException Thrown, if file channel closing or file deletion fail. 
-		 */
-		public void cleanup() throws IOException {
-			fileChannel.close();
-			if (!file.delete()) {
-				throw new IOException("Cannot remove temp file for stream alignment writer");
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.java
deleted file mode 100644
index 791fd40..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierHandler.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.runtime.io;
-
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import java.io.IOException;
-
-/**
- * The CheckpointBarrierHandler reacts to checkpoint barrier arriving from the input channels.
- * Different implementations may either simply track barriers, or block certain inputs on
- * barriers.
- */
-public interface CheckpointBarrierHandler {
-
-	/**
-	 * Returns the next {@link BufferOrEvent} that the operator may consume.
-	 * This call blocks until the next BufferOrEvent is available, ir until the stream
-	 * has been determined to be finished.
-	 * 
-	 * @return The next BufferOrEvent, or {@code null}, if the stream is finished.
-	 * @throws java.io.IOException Thrown, if the network or local disk I/O fails.
-	 * @throws java.lang.InterruptedException Thrown, if the thread is interrupted while blocking during
-	 *                                        waiting for the next BufferOrEvent to become available.
-	 */
-	BufferOrEvent getNextNonBlocked() throws IOException, InterruptedException;
-
-	/**
-	 * Registers the given event handler to be notified on successful checkpoints.
-	 * 
-	 * @param checkpointHandler The handler to register.
-	 */
-	void registerCheckpointEventHandler(EventListener<CheckpointBarrier> checkpointHandler);
-
-	/**
-	 * Cleans up all internally held resources.
-	 * 
-	 * @throws IOException Thrown, if the cleanup of I/O resources failed.
-	 */
-	void cleanup() throws IOException;
-
-	/**
-	 * Checks if the barrier handler has buffered any data internally.
-	 * @return True, if no data is buffered internally, false otherwise.
-	 */
-	boolean isEmpty();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java
deleted file mode 100644
index 01e997d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.io;
-
-import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-
-import java.util.ArrayList;
-
-public class CollectorWrapper<OUT> implements Output<StreamRecord<OUT>> {
-
-	private OutputSelectorWrapper<OUT> outputSelectorWrapper;
-
-	private ArrayList<Output<StreamRecord<OUT>>> allOutputs;
-
-	public CollectorWrapper(OutputSelectorWrapper<OUT> outputSelectorWrapper) {
-		this.outputSelectorWrapper = outputSelectorWrapper;
-		allOutputs = new ArrayList<Output<StreamRecord<OUT>>>();
-	}
-	
-	public void addCollector(Output<StreamRecord<OUT>> output, StreamEdge edge) {
-		outputSelectorWrapper.addCollector(output, edge);
-		allOutputs.add(output);
-	}
-
-	@Override
-	public void collect(StreamRecord<OUT> record) {
-		for (Collector<StreamRecord<OUT>> output : outputSelectorWrapper.getSelectedOutputs(record.getValue())) {
-			output.collect(record);
-		}
-	}
-
-	@Override
-	public void emitWatermark(Watermark mark) {
-		for (Output<?> output : allOutputs) {
-			output.emitWatermark(mark);
-		}
-	}
-
-	@Override
-	public void close() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateUtil.java
deleted file mode 100644
index f11e9a1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/InputGateUtil.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.io;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate;
-
-/**
- * Utility for dealing with input gates. This will either just return
- * the single {@link InputGate} that was passed in or create a {@link UnionInputGate} if several
- * {@link InputGate input gates} are given.
- */
-public class InputGateUtil {
-
-	public static InputGate createInputGate(Collection<InputGate> inputGates1, Collection<InputGate> inputGates2) {
-		List<InputGate> gates = new ArrayList<InputGate>(inputGates1.size() + inputGates2.size());
-		gates.addAll(inputGates1);
-		gates.addAll(inputGates2);
-		return createInputGate(gates.toArray(new InputGate[gates.size()]));
-	}
-
-	public static InputGate createInputGate(InputGate[] inputGates) {
-		if (inputGates.length <= 0) {
-			throw new RuntimeException("No such input gate.");
-		}
-
-		if (inputGates.length < 2) {
-			return inputGates[0];
-		} else {
-			return new UnionInputGate(inputGates);
-		}
-	}
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private InputGateUtil() {
-		throw new RuntimeException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
deleted file mode 100644
index 34e5800..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.io;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Implementation of {@link Output} that sends data using a {@link RecordWriter}.
- */
-public class RecordWriterOutput<OUT> implements Output<StreamRecord<OUT>> {
-
-	private StreamRecordWriter<SerializationDelegate<StreamElement>> recordWriter;
-	
-	private SerializationDelegate<StreamElement> serializationDelegate;
-
-	
-	@SuppressWarnings("unchecked")
-	public RecordWriterOutput(
-			StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>> recordWriter,
-			TypeSerializer<OUT> outSerializer,
-			boolean enableWatermarkMultiplexing) {
-
-		checkNotNull(recordWriter);
-		
-		// generic hack: cast the writer to generic Object type so we can use it 
-		// with multiplexed records and watermarks
-		this.recordWriter = (StreamRecordWriter<SerializationDelegate<StreamElement>>) 
-				(StreamRecordWriter<?>) recordWriter;
-
-		TypeSerializer<StreamElement> outRecordSerializer;
-		if (enableWatermarkMultiplexing) {
-			outRecordSerializer = new MultiplexingStreamRecordSerializer<OUT>(outSerializer);
-		} else {
-			outRecordSerializer = (TypeSerializer<StreamElement>)
-					(TypeSerializer<?>) new StreamRecordSerializer<OUT>(outSerializer);
-		}
-
-		if (outSerializer != null) {
-			serializationDelegate = new SerializationDelegate<StreamElement>(outRecordSerializer);
-		}
-	}
-
-	@Override
-	public void collect(StreamRecord<OUT> record) {
-		serializationDelegate.setInstance(record);
-
-		try {
-			recordWriter.emit(serializationDelegate);
-		}
-		catch (Exception e) {
-			throw new RuntimeException(e.getMessage(), e);
-		}
-	}
-
-	@Override
-	public void emitWatermark(Watermark mark) {
-		serializationDelegate.setInstance(mark);
-		
-		try {
-			recordWriter.broadcastEmit(serializationDelegate);
-		}
-		catch (Exception e) {
-			throw new RuntimeException(e.getMessage(), e);
-		}
-	}
-
-	public void broadcastEvent(AbstractEvent barrier) throws IOException, InterruptedException {
-		recordWriter.broadcastEvent(barrier);
-	}
-	
-	
-	public void flush() throws IOException {
-		recordWriter.flush();
-	}
-	
-	@Override
-	public void close() {
-		recordWriter.close();
-	}
-
-	public void clearBuffers() {
-		recordWriter.clearBuffers();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
deleted file mode 100644
index e131cda..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.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.io;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
-import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
-import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-/**
- * Input reader for {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}.
- *
- * <p>
- * This also keeps track of {@link Watermark} events and forwards them to event subscribers
- * once the {@link Watermark} from all inputs advances.
- *
- * <p>
- * Forwarding elements or watermarks must be protected by synchronizing on the given lock
- * object. This ensures that we don't call methods on a {@link OneInputStreamOperator} concurrently
- * with the timer callback or other things.
- * 
- * @param <IN> The type of the record that can be read with this record reader.
- */
-public class StreamInputProcessor<IN> {
-	
-	private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
-
-	private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
-
-	private final CheckpointBarrierHandler barrierHandler;
-
-	// We need to keep track of the channel from which a buffer came, so that we can
-	// appropriately map the watermarks to input channels
-	private int currentChannel = -1;
-
-	private boolean isFinished;
-
-	
-
-	private final long[] watermarks;
-	private long lastEmittedWatermark;
-
-	private final DeserializationDelegate<StreamElement> deserializationDelegate;
-
-	@SuppressWarnings("unchecked")
-	public StreamInputProcessor(InputGate[] inputGates, TypeSerializer<IN> inputSerializer,
-								EventListener<CheckpointBarrier> checkpointListener,
-								CheckpointingMode checkpointMode,
-								IOManager ioManager,
-								boolean enableWatermarkMultiplexing) throws IOException {
-
-		InputGate inputGate = InputGateUtil.createInputGate(inputGates);
-
-		if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
-			this.barrierHandler = new BarrierBuffer(inputGate, ioManager);
-		}
-		else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) {
-			this.barrierHandler = new BarrierTracker(inputGate);
-		}
-		else {
-			throw new IllegalArgumentException("Unrecognized CheckpointingMode: " + checkpointMode);
-		}
-		
-		if (checkpointListener != null) {
-			this.barrierHandler.registerCheckpointEventHandler(checkpointListener);
-		}
-		
-		if (enableWatermarkMultiplexing) {
-			MultiplexingStreamRecordSerializer<IN> ser = new MultiplexingStreamRecordSerializer<IN>(inputSerializer);
-			this.deserializationDelegate = new NonReusingDeserializationDelegate<StreamElement>(ser);
-		} else {
-			StreamRecordSerializer<IN> ser = new StreamRecordSerializer<IN>(inputSerializer);
-			this.deserializationDelegate = (NonReusingDeserializationDelegate<StreamElement>)
-					(NonReusingDeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN>>(ser);
-		}
-		
-		// Initialize one deserializer per input channel
-		this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()];
-		
-		for (int i = 0; i < recordDeserializers.length; i++) {
-			recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<StreamElement>>();
-		}
-
-		watermarks = new long[inputGate.getNumberOfInputChannels()];
-		for (int i = 0; i < inputGate.getNumberOfInputChannels(); i++) {
-			watermarks[i] = Long.MIN_VALUE;
-		}
-		lastEmittedWatermark = Long.MIN_VALUE;
-	}
-
-	@SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
-	public boolean processInput(OneInputStreamOperator<IN, ?> streamOperator, final Object lock) throws Exception {
-		if (isFinished) {
-			return false;
-		}
-
-		while (true) {
-			if (currentRecordDeserializer != null) {
-				DeserializationResult result = currentRecordDeserializer.getNextRecord(deserializationDelegate);
-
-				if (result.isBufferConsumed()) {
-					currentRecordDeserializer.getCurrentBuffer().recycle();
-					currentRecordDeserializer = null;
-				}
-
-				if (result.isFullRecord()) {
-					StreamElement recordOrWatermark = deserializationDelegate.getInstance();
-
-					if (recordOrWatermark.isWatermark()) {
-						long watermarkMillis = recordOrWatermark.asWatermark().getTimestamp();
-						if (watermarkMillis > watermarks[currentChannel]) {
-							watermarks[currentChannel] = watermarkMillis;
-							long newMinWatermark = Long.MAX_VALUE;
-							for (long watermark : watermarks) {
-								newMinWatermark = Math.min(watermark, newMinWatermark);
-							}
-							if (newMinWatermark > lastEmittedWatermark) {
-								lastEmittedWatermark = newMinWatermark;
-								synchronized (lock) {
-									streamOperator.processWatermark(new Watermark(lastEmittedWatermark));
-								}
-							}
-						}
-						continue;
-					} else {
-						// now we can do the actual processing
-						StreamRecord<IN> record = recordOrWatermark.asRecord();
-						synchronized (lock) {
-							streamOperator.setKeyContextElement(record);
-							streamOperator.processElement(record);
-						}
-						return true;
-					}
-				}
-			}
-
-			final BufferOrEvent bufferOrEvent = barrierHandler.getNextNonBlocked();
-			if (bufferOrEvent != null) {
-				if (bufferOrEvent.isBuffer()) {
-					currentChannel = bufferOrEvent.getChannelIndex();
-					currentRecordDeserializer = recordDeserializers[currentChannel];
-					currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer());
-				}
-				else {
-					// Event received
-					final AbstractEvent event = bufferOrEvent.getEvent();
-					if (event.getClass() != EndOfPartitionEvent.class) {
-						throw new IOException("Unexpected event: " + event);
-					}
-				}
-			}
-			else {
-				isFinished = true;
-				if (!barrierHandler.isEmpty()) {
-					throw new IllegalStateException("Trailing data in checkpoint barrier handler.");
-				}
-				return false;
-			}
-		}
-	}
-	
-	public void setReporter(AccumulatorRegistry.Reporter reporter) {
-		for (RecordDeserializer<?> deserializer : recordDeserializers) {
-			deserializer.setReporter(reporter);
-		}
-	}
-	
-	public void cleanup() throws IOException {
-		// clear the buffers first. this part should not ever fail
-		for (RecordDeserializer<?> deserializer : recordDeserializers) {
-			Buffer buffer = deserializer.getCurrentBuffer();
-			if (buffer != null && !buffer.isRecycled()) {
-				buffer.recycle();
-			}
-		}
-		
-		// cleanup the barrier handler resources
-		barrierHandler.cleanup();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java
deleted file mode 100644
index 8dcaad8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamRecordWriter.java
+++ /dev/null
@@ -1,182 +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.io;
-
-import java.io.IOException;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
-import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-/**
- * This record writer keeps data in buffers at most for a certain timeout. It spawns a separate thread
- * that flushes the outputs in a defined interval, to make sure data does not linger in the buffers for too long.
- * 
- * @param <T> The type of elements written.
- */
-public class StreamRecordWriter<T extends IOReadableWritable> extends RecordWriter<T> {
-
-	/** Default name for teh output flush thread, if no name with a task reference is given */
-	private static final String DEFAULT_OUTPUT_FLUSH_THREAD_NAME = "OutputFlusher";
-	
-	
-	/** The thread that periodically flushes the output, to give an upper latency bound */
-	private final OutputFlusher outputFlusher;
-	
-	/** Flag indicating whether the output should be flushed after every element */
-	private final boolean flushAlways;
-
-	/** The exception encountered in the flushing thread */
-	private Throwable flusherException;
-	
-	
-	
-	public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSelector, long timeout) {
-		this(writer, channelSelector, timeout, null);
-	}
-	
-	public StreamRecordWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSelector,
-								long timeout, String taskName) {
-		
-		super(writer, channelSelector);
-		
-		checkArgument(timeout >= -1);
-		
-		if (timeout == -1) {
-			flushAlways = false;
-			outputFlusher = null;
-		}
-		else if (timeout == 0) {
-			flushAlways = true;
-			outputFlusher = null;
-		}
-		else {
-			flushAlways = false;
-			String threadName = taskName == null ?
-								DEFAULT_OUTPUT_FLUSH_THREAD_NAME : "Output Timeout Flusher - " + taskName;
-			
-			outputFlusher = new OutputFlusher(threadName, timeout);
-			outputFlusher.start();
-		}
-	}
-	
-	@Override
-	public void emit(T record) throws IOException, InterruptedException {
-		checkErroneous();
-		super.emit(record);
-		if (flushAlways) {
-			flush();
-		}
-	}
-
-	@Override
-	public void broadcastEmit(T record) throws IOException, InterruptedException {
-		checkErroneous();
-		super.broadcastEmit(record);
-		if (flushAlways) {
-			flush();
-		}
-	}
-
-	/**
-	 * Closes the writer. This stops the flushing thread (if there is one).
-	 */
-	public void close() {
-		// make sure we terminate the thread in any case
-		if (outputFlusher != null) {
-			outputFlusher.terminate();
-			try {
-				outputFlusher.join();
-			}
-			catch (InterruptedException e) {
-				// ignore on close
-			}
-		}
-	}
-
-	/**
-	 * Notifies the writer that the output flusher thread encountered an exception.
-	 * 
-	 * @param t The exception to report.
-	 */
-	private void notifyFlusherException(Throwable t) {
-		if (this.flusherException == null) {
-			this.flusherException = t;
-		}
-	}
-	
-	private void checkErroneous() throws IOException {
-		if (flusherException != null) {
-			throw new IOException("An exception happened while flushing the outputs", flusherException);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * A dedicated thread that periodically flushes the output buffers, to set upper latency bounds.
-	 * 
-	 * The thread is daemonic, because it is only a utility thread.
-	 */
-	private class OutputFlusher extends Thread {
-		
-		private final long timeout;
-		
-		private volatile boolean running = true;
-
-		
-		OutputFlusher(String name, long timeout) {
-			super(name);
-			setDaemon(true);
-			this.timeout = timeout;
-		}
-		
-		public void terminate() {
-			running = false;
-			interrupt();
-		}
-
-		@Override
-		public void run() {
-			try {
-				while (running) {
-					try {
-						Thread.sleep(timeout);
-					}
-					catch (InterruptedException e) {
-						// propagate this if we are still running, because it should not happen
-						// in that case
-						if (running) {
-							throw new Exception(e);
-						}
-					}
-					
-					// any errors here should let the thread come to a halt and be
-					// recognized by the writer 
-					flush();
-				}
-			}
-			catch (Throwable t) {
-				notifyFlusherException(t);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
deleted file mode 100644
index 882037e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+++ /dev/null
@@ -1,290 +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.io;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
-import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
-import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.plugable.DeserializationDelegate;
-import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-
-/**
- * Input reader for {@link org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask}.
- *
- * <p>
- * This also keeps track of {@link org.apache.flink.streaming.api.watermark.Watermark} events and forwards them to event subscribers
- * once the {@link org.apache.flink.streaming.api.watermark.Watermark} from all inputs advances.
- *
- * <p>
- * Forwarding elements or watermarks must be protected by synchronizing on the given lock
- * object. This ensures that we don't call methods on a {@link TwoInputStreamOperator} concurrently
- * with the timer callback or other things.
- *
- * @param <IN1> The type of the records that arrive on the first input
- * @param <IN2> The type of the records that arrive on the second input
- */
-public class StreamTwoInputProcessor<IN1, IN2> {
-
-	private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
-
-	private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
-
-	// We need to keep track of the channel from which a buffer came, so that we can
-	// appropriately map the watermarks to input channels
-	private int currentChannel = -1;
-
-	private boolean isFinished;
-
-	private final CheckpointBarrierHandler barrierHandler;
-
-	private final long[] watermarks1;
-	private long lastEmittedWatermark1;
-
-	private final long[] watermarks2;
-	private long lastEmittedWatermark2;
-
-	private final int numInputChannels1;
-
-	private final DeserializationDelegate<StreamElement> deserializationDelegate1;
-	private final DeserializationDelegate<StreamElement> deserializationDelegate2;
-
-	@SuppressWarnings({"unchecked", "rawtypes"})
-	public StreamTwoInputProcessor(
-			Collection<InputGate> inputGates1,
-			Collection<InputGate> inputGates2,
-			TypeSerializer<IN1> inputSerializer1,
-			TypeSerializer<IN2> inputSerializer2,
-			EventListener<CheckpointBarrier> checkpointListener,
-			CheckpointingMode checkpointMode,
-			IOManager ioManager,
-			boolean enableWatermarkMultiplexing) throws IOException {
-		
-		final InputGate inputGate = InputGateUtil.createInputGate(inputGates1, inputGates2);
-
-		if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
-			this.barrierHandler = new BarrierBuffer(inputGate, ioManager);
-		}
-		else if (checkpointMode == CheckpointingMode.AT_LEAST_ONCE) {
-			this.barrierHandler = new BarrierTracker(inputGate);
-		}
-		else {
-			throw new IllegalArgumentException("Unrecognized CheckpointingMode: " + checkpointMode);
-		}
-		
-		if (checkpointListener != null) {
-			this.barrierHandler.registerCheckpointEventHandler(checkpointListener);
-		}
-		
-		if (enableWatermarkMultiplexing) {
-			MultiplexingStreamRecordSerializer<IN1> ser = new MultiplexingStreamRecordSerializer<IN1>(inputSerializer1);
-			this.deserializationDelegate1 = new NonReusingDeserializationDelegate<StreamElement>(ser);
-		}
-		else {
-			StreamRecordSerializer<IN1> ser = new StreamRecordSerializer<IN1>(inputSerializer1);
-			this.deserializationDelegate1 = (DeserializationDelegate<StreamElement>)
-					(DeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN1>>(ser);
-		}
-		
-		if (enableWatermarkMultiplexing) {
-			MultiplexingStreamRecordSerializer<IN2> ser = new MultiplexingStreamRecordSerializer<IN2>(inputSerializer2);
-			this.deserializationDelegate2 = new NonReusingDeserializationDelegate<StreamElement>(ser);
-		}
-		else {
-			StreamRecordSerializer<IN2> ser = new StreamRecordSerializer<IN2>(inputSerializer2);
-			this.deserializationDelegate2 = (DeserializationDelegate<StreamElement>)
-					(DeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN2>>(ser);
-		}
-
-		// Initialize one deserializer per input channel
-		this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()];
-		
-		for (int i = 0; i < recordDeserializers.length; i++) {
-			recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<StreamElement>>();
-		}
-
-		// determine which unioned channels belong to input 1 and which belong to input 2
-		int numInputChannels1 = 0;
-		for (InputGate gate: inputGates1) {
-			numInputChannels1 += gate.getNumberOfInputChannels();
-		}
-		
-		this.numInputChannels1 = numInputChannels1;
-		int numInputChannels2 = inputGate.getNumberOfInputChannels() - numInputChannels1;
-
-		watermarks1 = new long[numInputChannels1];
-		Arrays.fill(watermarks1, Long.MIN_VALUE);
-		lastEmittedWatermark1 = Long.MIN_VALUE;
-
-		watermarks2 = new long[numInputChannels2];
-		Arrays.fill(watermarks2, Long.MIN_VALUE);
-		lastEmittedWatermark2 = Long.MIN_VALUE;
-	}
-
-	@SuppressWarnings("unchecked")
-	public boolean processInput(TwoInputStreamOperator<IN1, IN2, ?> streamOperator, Object lock) throws Exception {
-		if (isFinished) {
-			return false;
-		}
-
-		while (true) {
-			if (currentRecordDeserializer != null) {
-				DeserializationResult result;
-				if (currentChannel < numInputChannels1) {
-					result = currentRecordDeserializer.getNextRecord(deserializationDelegate1);
-				} else {
-					result = currentRecordDeserializer.getNextRecord(deserializationDelegate2);
-				}
-
-				if (result.isBufferConsumed()) {
-					currentRecordDeserializer.getCurrentBuffer().recycle();
-					currentRecordDeserializer = null;
-				}
-
-				if (result.isFullRecord()) {
-					if (currentChannel < numInputChannels1) {
-						StreamElement recordOrWatermark = deserializationDelegate1.getInstance();
-						if (recordOrWatermark.isWatermark()) {
-							handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel, lock);
-							continue;
-						}
-						else {
-							synchronized (lock) {
-								streamOperator.processElement1(recordOrWatermark.<IN1>asRecord());
-							}
-							return true;
-
-						}
-					}
-					else {
-						StreamElement recordOrWatermark = deserializationDelegate2.getInstance();
-						if (recordOrWatermark.isWatermark()) {
-							handleWatermark(streamOperator, recordOrWatermark.asWatermark(), currentChannel, lock);
-							continue;
-						}
-						else {
-							synchronized (lock) {
-								streamOperator.processElement2(recordOrWatermark.<IN2>asRecord());
-							}
-							return true;
-						}
-					}
-				}
-			}
-
-			final BufferOrEvent bufferOrEvent = barrierHandler.getNextNonBlocked();
-			if (bufferOrEvent != null) {
-
-				if (bufferOrEvent.isBuffer()) {
-					currentChannel = bufferOrEvent.getChannelIndex();
-					currentRecordDeserializer = recordDeserializers[currentChannel];
-					currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer());
-	
-				} else {
-					// Event received
-					final AbstractEvent event = bufferOrEvent.getEvent();
-					if (event.getClass() != EndOfPartitionEvent.class) {
-						throw new IOException("Unexpected event: " + event);
-					}
-				}
-			}
-			else {
-				isFinished = true;
-				if (!barrierHandler.isEmpty()) {
-					throw new IllegalStateException("Trailing data in checkpoint barrier handler.");
-				}
-				return false;
-			}
-		}
-	}
-
-	private void handleWatermark(TwoInputStreamOperator<IN1, IN2, ?> operator, Watermark mark, int channelIndex, Object lock) throws Exception {
-		if (channelIndex < numInputChannels1) {
-			long watermarkMillis = mark.getTimestamp();
-			if (watermarkMillis > watermarks1[channelIndex]) {
-				watermarks1[channelIndex] = watermarkMillis;
-				long newMinWatermark = Long.MAX_VALUE;
-				for (long wm : watermarks1) {
-					newMinWatermark = Math.min(wm, newMinWatermark);
-				}
-				if (newMinWatermark > lastEmittedWatermark1) {
-					lastEmittedWatermark1 = newMinWatermark;
-					synchronized (lock) {
-						operator.processWatermark1(new Watermark(lastEmittedWatermark1));
-					}
-				}
-			}
-		} else {
-			channelIndex = channelIndex - numInputChannels1;
-			long watermarkMillis = mark.getTimestamp();
-			if (watermarkMillis > watermarks2[channelIndex]) {
-				watermarks2[channelIndex] = watermarkMillis;
-				long newMinWatermark = Long.MAX_VALUE;
-				for (long wm : watermarks2) {
-					newMinWatermark = Math.min(wm, newMinWatermark);
-				}
-				if (newMinWatermark > lastEmittedWatermark2) {
-					lastEmittedWatermark2 = newMinWatermark;
-					synchronized (lock) {
-						operator.processWatermark2(new Watermark(lastEmittedWatermark2));
-					}
-				}
-			}
-		}
-	}
-	
-	public void setReporter(AccumulatorRegistry.Reporter reporter) {
-		for (RecordDeserializer<?> deserializer : recordDeserializers) {
-			deserializer.setReporter(reporter);
-		}
-	}
-	
-	public void cleanup() throws IOException {
-		// clear the buffers first. this part should not ever fail
-		for (RecordDeserializer<?> deserializer : recordDeserializers) {
-			Buffer buffer = deserializer.getCurrentBuffer();
-			if (buffer != null && !buffer.isRecycled()) {
-				buffer.recycle();
-			}
-		}
-
-		// cleanup the barrier handler resources
-		barrierHandler.cleanup();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingReader.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingReader.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingReader.java
deleted file mode 100644
index 793e87e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamingReader.java
+++ /dev/null
@@ -1,28 +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.io;
-
-import org.apache.flink.runtime.io.network.api.reader.ReaderBase;
-
-import java.io.IOException;
-
-public interface StreamingReader extends ReaderBase {
-
-	void cleanup() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
deleted file mode 100644
index 7020758..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
+++ /dev/null
@@ -1,103 +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;
-
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-
-/**
- * An operator that can sort a stream based on timestamps. Arriving elements will be put into
- * buckets based on their timestamp. Sorting and emission of sorted elements happens once
- * the watermark passes the end of a bucket.
- *
- * @param <T> The type of the elements on which this operator works.
- */
-public class BucketStreamSortOperator<T> extends AbstractStreamOperator<T> implements OneInputStreamOperator<T, T> {
-	private static final long serialVersionUID = 1L;
-
-	private long granularity;
-
-	private transient Map<Long, List<StreamRecord<T>>> buckets;
-
-	/**
-	 * Creates a new sorting operator that creates buckets with the given interval.
-	 *
-	 * @param interval The size (in time) of one bucket.
-	 */
-	public BucketStreamSortOperator(long interval) {
-		this.granularity = interval;
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-		buckets = new HashMap<>();
-
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public void processElement(StreamRecord<T> record) throws Exception {
-		long bucketId = record.getTimestamp() - (record.getTimestamp() % granularity);
-		List<StreamRecord<T>> bucket = buckets.get(bucketId);
-		if (bucket == null) {
-			bucket = new ArrayList<>();
-			buckets.put(bucketId, bucket);
-		}
-		bucket.add(record);
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		long maxBucketId = mark.getTimestamp() - (mark.getTimestamp() % granularity);
-		Set<Long> toRemove = new HashSet<>();
-		for (Map.Entry<Long, List<StreamRecord<T>>> bucket: buckets.entrySet()) {
-			if (bucket.getKey() < maxBucketId) {
-				Collections.sort(bucket.getValue(), new Comparator<StreamRecord<T>>() {
-					@Override
-					public int compare(StreamRecord<T> o1, StreamRecord<T> o2) {
-						return (int) (o1.getTimestamp() - o2.getTimestamp());
-					}
-				});
-				for (StreamRecord<T> r: bucket.getValue()) {
-					output.collect(r);
-				}
-				toRemove.add(bucket.getKey());
-			}
-		}
-
-		for (Long l: toRemove) {
-			buckets.remove(l);
-		}
-
-		output.emitWatermark(mark);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java
deleted file mode 100644
index 6e51a49..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java
+++ /dev/null
@@ -1,95 +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;
-
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * A {@link org.apache.flink.streaming.api.operators.StreamOperator} for extracting timestamps
- * from user elements and assigning them as the internal timestamp of the {@link StreamRecord}.
- *
- * @param <T> The type of the input elements
- */
-public class ExtractTimestampsOperator<T>
-		extends AbstractUdfStreamOperator<T, TimestampExtractor<T>>
-		implements OneInputStreamOperator<T, T>, Triggerable {
-
-	private static final long serialVersionUID = 1L;
-
-	transient long watermarkInterval;
-
-	transient long currentWatermark;
-
-	public ExtractTimestampsOperator(TimestampExtractor<T> extractor) {
-		super(extractor);
-		chainingStrategy = ChainingStrategy.ALWAYS;
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-		watermarkInterval = getExecutionConfig().getAutoWatermarkInterval();
-		if (watermarkInterval > 0) {
-			registerTimer(System.currentTimeMillis() + watermarkInterval, this);
-		}
-
-		currentWatermark = Long.MIN_VALUE;
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-
-		// emit a final +Inf watermark, just like the sources
-		output.emitWatermark(new Watermark(Long.MAX_VALUE));
-	}
-
-	@Override
-	public void processElement(StreamRecord<T> element) throws Exception {
-		long newTimestamp = userFunction.extractTimestamp(element.getValue(), element.getTimestamp());
-		output.collect(element.replace(element.getValue(), newTimestamp));
-		long watermark = userFunction.extractWatermark(element.getValue(), newTimestamp);
-		if (watermark > currentWatermark) {
-			currentWatermark = watermark;
-			output.emitWatermark(new Watermark(currentWatermark));
-		}
-	}
-
-	@Override
-	public void trigger(long timestamp) throws Exception {
-		// register next timer
-		registerTimer(System.currentTimeMillis() + watermarkInterval, this);
-		long lastWatermark = currentWatermark;
-		currentWatermark = userFunction.getCurrentWatermark();
-
-		if (currentWatermark > lastWatermark) {
-			// emit watermark
-			output.emitWatermark(new Watermark(currentWatermark));
-		}
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		// ignore them, since we are basically a watermark source
-	}
-}


[40/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 98506e0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ /dev/null
@@ -1,1454 +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.environment;
-
-import com.esotericsoftware.kryo.Serializer;
-import com.google.common.base.Preconditions;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.io.FileInputFormat;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.PrimitiveInputFormat;
-import org.apache.flink.api.java.io.TextInputFormat;
-import org.apache.flink.api.java.io.TextValueInputFormat;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.MissingTypeInfo;
-import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.ValueTypeInfo;
-import org.apache.flink.client.program.Client;
-import org.apache.flink.client.program.ContextEnvironment;
-import org.apache.flink.client.program.OptimizerPlanEnvironment;
-import org.apache.flink.client.program.PreviewPlanEnvironment;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction;
-import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType;
-import org.apache.flink.streaming.api.functions.source.FileReadFunction;
-import org.apache.flink.streaming.api.functions.source.FileSourceFunction;
-import org.apache.flink.streaming.api.functions.source.FromElementsFunction;
-import org.apache.flink.streaming.api.functions.source.FromIteratorFunction;
-import org.apache.flink.streaming.api.functions.source.FromSplittableIteratorFunction;
-import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-import org.apache.flink.types.StringValue;
-import org.apache.flink.util.SplittableIterator;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * 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";
-	
-	/** The time characteristic that is used if none other is set */
-	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();
-	
-	// ------------------------------------------------------------------------
-
-	/** 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
-
-	protected CheckpointingMode checkpointingMode;
-
-	protected boolean forceCheckpointing = false;
-	
-	/** The state backend used for storing k/v state and state snapshots */
-	private StateBackend<?> defaultStateBackend;
-	
-	/** The time characteristic used by the data streams */
-	private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC;
-
-	
-	// --------------------------------------------------------------------------------------------
-	// Constructor and Properties
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Gets the config object.
-	 */
-	public ExecutionConfig getConfig() {
-		return config;
-	}
-
-	/**
-	 * 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) {
-			throw new IllegalArgumentException("parallelism must be at least one.");
-		}
-		config.setParallelism(parallelism);
-		return this;
-	}
-
-	/**
-	 * 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.
-	 */
-	public int getParallelism() {
-		return config.getParallelism();
-	}
-
-	/**
-	 * Sets the maximum time frequency (milliseconds) for the flushing of the
-	 * output buffers. By default the output buffers flush frequently to provide
-	 * low latency and to aid smooth developer experience. Setting the parameter
-	 * can result in three logical modes:
-	 * <p/>
-	 * <ul>
-	 * <li>
-	 * A positive integer triggers flushing periodically by that integer</li>
-	 * <li>
-	 * 0 triggers flushing after every record thus minimizing latency</li>
-	 * <li>
-	 * -1 triggers flushing only when the output buffer is full thus maximizing
-	 * throughput</li>
-	 * </ul>
-	 *
-	 * @param timeoutMillis
-	 * 		The maximum time between two output flushes.
-	 */
-	public StreamExecutionEnvironment setBufferTimeout(long timeoutMillis) {
-		if (timeoutMillis < -1) {
-			throw new IllegalArgumentException("Timeout of buffer must be non-negative or -1");
-		}
-
-		this.bufferTimeout = timeoutMillis;
-		return this;
-	}
-
-	/**
-	 * Sets the maximum time frequency (milliseconds) for the flushing of the
-	 * output buffers. For clarification on the extremal values see
-	 * {@link #setBufferTimeout(long)}.
-	 *
-	 * @return The timeout of the buffer.
-	 */
-	public long getBufferTimeout() {
-		return this.bufferTimeout;
-	}
-
-	/**
-	 * Disables operator chaining for streaming operators. Operator chaining
-	 * allows non-shuffle operations to be co-located in the same thread fully
-	 * avoiding serialization and de-serialization.
-	 *
-	 * @return StreamExecutionEnvironment with chaining disabled.
-	 */
-	public StreamExecutionEnvironment disableOperatorChaining() {
-		this.isChainingEnabled = false;
-		return this;
-	}
-
-	/**
-	 * Returns whether operator chaining is enabled.
-	 *
-	 * @return {@code true} if chaining is enabled, false otherwise.
-	 */
-	public boolean isChainingEnabled() {
-		return isChainingEnabled;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpointing Settings
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Enables checkpointing for the streaming job. The distributed state of the streaming
-	 * dataflow will be periodically snapshotted. In case of a failure, the streaming
-	 * dataflow will be restarted from the latest completed checkpoint. This method selects
-	 * {@link CheckpointingMode#EXACTLY_ONCE} guarantees.
-	 * 
-	 * <p>The job draws checkpoints periodically, in the given interval. The state will be
-	 * stored in the configured state backend.</p>
-	 * 
-	 * <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at
-	 * the moment. For that reason, iterative jobs will not be started if used
-	 * with enabled checkpointing. To override this mechanism, use the 
-	 * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.</p>
-	 *
-	 * @param interval Time interval between state checkpoints in milliseconds.
-	 */
-	public StreamExecutionEnvironment enableCheckpointing(long interval) {
-		return enableCheckpointing(interval, CheckpointingMode.EXACTLY_ONCE);
-	}
-
-	/**
-	 * Enables checkpointing for the streaming job. The distributed state of the streaming
-	 * dataflow will be periodically snapshotted. In case of a failure, the streaming
-	 * dataflow will be restarted from the latest completed checkpoint.
-	 *
-	 * <p>The job draws checkpoints periodically, in the given interval. The system uses the
-	 * given {@link CheckpointingMode} for the checkpointing ("exactly once" vs "at least once").
-	 * The state will be stored in the configured state backend.</p>
-	 *
-	 * <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at
-	 * the moment. For that reason, iterative jobs will not be started if used
-	 * with enabled checkpointing. To override this mechanism, use the 
-	 * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.</p>
-	 *
-	 * @param interval 
-	 *             Time interval between state checkpoints in milliseconds.
-	 * @param mode 
-	 *             The checkpointing mode, selecting between "exactly once" and "at least once" guaranteed.
-	 */
-	public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode) {
-		if (mode == null) {
-			throw new NullPointerException("checkpoint mode must not be null");
-		}
-		if (interval <= 0) {
-			throw new IllegalArgumentException("the checkpoint interval must be positive");
-		}
-
-		this.checkpointInterval = interval;
-		this.checkpointingMode = mode;
-		return this;
-	}
-	
-	/**
-	 * Enables checkpointing for the streaming job. The distributed state of the streaming
-	 * dataflow will be periodically snapshotted. In case of a failure, the streaming
-	 * dataflow will be restarted from the latest completed checkpoint.
-	 *
-	 * <p>The job draws checkpoints periodically, in the given interval. The state will be
-	 * stored in the configured state backend.</p>
-	 *
-	 * <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at
-	 * the moment. If the "force" parameter is set to true, the system will execute the
-	 * job nonetheless.</p>
-	 * 
-	 * @param interval
-	 *            Time interval between state checkpoints in millis.
-	 * @param mode
-	 *            The checkpointing mode, selecting between "exactly once" and "at least once" guaranteed.
-	 * @param force
-	 *            If true checkpointing will be enabled for iterative jobs as well.
-	 */
-	@Deprecated
-	public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode, boolean force) {
-		this.enableCheckpointing(interval, mode);
-
-		this.forceCheckpointing = force;
-		return this;
-	}
-
-	/**
-	 * Enables checkpointing for the streaming job. The distributed state of the streaming
-	 * dataflow will be periodically snapshotted. In case of a failure, the streaming
-	 * dataflow will be restarted from the latest completed checkpoint. This method selects
-	 * {@link CheckpointingMode#EXACTLY_ONCE} guarantees.
-	 *
-	 * <p>The job draws checkpoints periodically, in the default interval. The state will be
-	 * stored in the configured state backend.</p>
-	 *
-	 * <p>NOTE: Checkpointing iterative streaming dataflows in not properly supported at
-	 * the moment. For that reason, iterative jobs will not be started if used
-	 * with enabled checkpointing. To override this mechanism, use the 
-	 * {@link #enableCheckpointing(long, CheckpointingMode, boolean)} method.</p>
-	 */
-	public StreamExecutionEnvironment enableCheckpointing() {
-		enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE);
-		return this;
-	}
-
-	/**
-	 * Returns the checkpointing interval or -1 if checkpointing is disabled.
-	 *
-	 * @return The checkpointing interval or -1
-	 */
-	public long getCheckpointInterval() {
-		return checkpointInterval;
-	}
-
-
-	/**
-	 * Returns whether checkpointing is force-enabled.
-	 */
-	public boolean isForceCheckpointing() {
-		return forceCheckpointing;
-	}
-
-	/**
-	 * Returns the {@link CheckpointingMode}.
-	 */
-	public CheckpointingMode getCheckpointingMode() {
-		return checkpointingMode;
-	}
-
-	/**
-	 * Sets the state backend that describes how to store and checkpoint operator state. It defines in
-	 * what form the key/value state ({@link org.apache.flink.api.common.state.OperatorState}, accessible
-	 * from operations on {@link org.apache.flink.streaming.api.datastream.KeyedStream}) is maintained
-	 * (heap, managed memory, externally), and where state snapshots/checkpoints are stored, both for
-	 * the key/value state, and for checkpointed functions (implementing the interface
-	 * {@link org.apache.flink.streaming.api.checkpoint.Checkpointed}).
-	 *
-	 * <p>The {@link org.apache.flink.runtime.state.memory.MemoryStateBackend} for example
-	 * maintains the state in heap memory, as objects. It is lightweight without extra dependencies,
-	 * but can checkpoint only small states (some counters).
-	 * 
-	 * <p>In contrast, the {@link org.apache.flink.runtime.state.filesystem.FsStateBackend}
-	 * stores checkpoints of the state (also maintained as heap objects) in files. When using a replicated
-	 * file system (like HDFS, S3, MapR FS, Tachyon, etc) this will guarantee that state is not lost upon
-	 * failures of individual nodes and that streaming program can be executed highly available and strongly
-	 * consistent (assuming that Flink is run in high-availability mode).
-	 *
-	 * @return This StreamExecutionEnvironment itself, to allow chaining of function calls.
-	 * 
-	 * @see #getStateBackend()
-	 */
-	public StreamExecutionEnvironment setStateBackend(StateBackend<?> backend) {
-		this.defaultStateBackend = requireNonNull(backend);
-		return this;
-	}
-
-	/**
-	 * Returns the state backend that defines how to store and checkpoint state.
-	 * @return The state backend that defines how to store and checkpoint state.
-	 * 
-	 * @see #setStateBackend(StateBackend)
-	 */
-	public StateBackend<?> getStateBackend() {
-		return defaultStateBackend;
-	}
-
-	/**
-	 * Sets the number of times that failed tasks are re-executed. A value of
-	 * zero effectively disables fault tolerance. A value of {@code -1}
-	 * indicates that the system default value (as defined in the configuration)
-	 * should be used.
-	 *
-	 * @param numberOfExecutionRetries
-	 * 		The number of times the system will try to re-execute failed tasks.
-	 */
-	public void setNumberOfExecutionRetries(int numberOfExecutionRetries) {
-		config.setNumberOfExecutionRetries(numberOfExecutionRetries);
-	}
-
-	/**
-	 * Gets the number of times the system will try to re-execute failed tasks.
-	 * A value of {@code -1} indicates that the system default value (as defined
-	 * in the configuration) should be used.
-	 *
-	 * @return The number of times the system will try to re-execute failed tasks.
-	 */
-	public int getNumberOfExecutionRetries() {
-		return config.getNumberOfExecutionRetries();
-	}
-
-	/**
-	 * Sets the delay that failed tasks are re-executed. A value of
-	 * zero effectively disables fault tolerance. A value of {@code -1}
-	 * indicates that the system default value (as defined in the configuration)
-	 * should be used.
-	 *
-	 * @param executionRetryDelay
-	 * 		The delay of time the system will wait to re-execute failed
-	 * 		tasks.
-	 */
-	public void setExecutionRetryDelay(long executionRetryDelay) {
-		config.setExecutionRetryDelay(executionRetryDelay);
-	}
-	
-	/**
-	 * Gets the delay time in milliseconds the system will wait to re-execute failed tasks.
-	 * A value of {@code -1} indicates that the system default value (as defined
-	 * in the configuration) should be used.
-	 *
-	 * @return The delay time the system will wait to re-execute failed tasks.
-	 */
-	public long getExecutionRetryDelay() {
-		return config.getExecutionRetryDelay();
-	}
-	/**
-	 * Sets the default parallelism that will be used for the local execution
-	 * environment created by {@link #createLocalEnvironment()}.
-	 *
-	 * @param parallelism
-	 * 		The parallelism to use as the default local parallelism.
-	 */
-	public static void setDefaultLocalParallelism(int parallelism) {
-		defaultLocalParallelism = parallelism;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// Registry for types and serializers
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Adds a new Kryo default serializer to the Runtime.
-	 * <p/>
-	 * Note that the serializer instance must be serializable (as defined by
-	 * java.io.Serializable), because it may be distributed to the worker nodes
-	 * by java serialization.
-	 *
-	 * @param type
-	 * 		The class of the types serialized with the given serializer.
-	 * @param serializer
-	 * 		The serializer to use.
-	 */
-	public <T extends Serializer<?> & Serializable>void addDefaultKryoSerializer(Class<?> type, T serializer) {
-		config.addDefaultKryoSerializer(type, serializer);
-	}
-
-	/**
-	 * Adds a new Kryo default serializer to the Runtime.
-	 *
-	 * @param type
-	 * 		The class of the types serialized with the given serializer.
-	 * @param serializerClass
-	 * 		The class of the serializer to use.
-	 */
-	public void addDefaultKryoSerializer(Class<?> type,
-			Class<? extends Serializer<?>> serializerClass) {
-		config.addDefaultKryoSerializer(type, serializerClass);
-	}
-
-	/**
-	 * Registers the given type with a Kryo Serializer.
-	 * <p/>
-	 * Note that the serializer instance must be serializable (as defined by
-	 * java.io.Serializable), because it may be distributed to the worker nodes
-	 * by java serialization.
-	 *
-	 * @param type
-	 * 		The class of the types serialized with the given serializer.
-	 * @param serializer
-	 * 		The serializer to use.
-	 */
-	public <T extends Serializer<?> & Serializable>void registerTypeWithKryoSerializer(Class<?> type, T serializer) {
-		config.registerTypeWithKryoSerializer(type, serializer);
-	}
-
-	/**
-	 * Registers the given Serializer via its class as a serializer for the
-	 * given type at the KryoSerializer
-	 *
-	 * @param type
-	 * 		The class of the types serialized with the given serializer.
-	 * @param serializerClass
-	 * 		The class of the serializer to use.
-	 */
-	public void registerTypeWithKryoSerializer(Class<?> type,
-			Class<? extends Serializer<?>> serializerClass) {
-		config.registerTypeWithKryoSerializer(type, serializerClass);
-	}
-
-	/**
-	 * Registers the given type with the serialization stack. If the type is
-	 * eventually serialized as a POJO, then the type is registered with the
-	 * POJO serializer. If the type ends up being serialized with Kryo, then it
-	 * will be registered at Kryo to make sure that only tags are written.
-	 *
-	 * @param type
-	 * 		The class of the type to register.
-	 */
-	public void registerType(Class<?> type) {
-		if (type == null) {
-			throw new NullPointerException("Cannot register null type class.");
-		}
-
-		TypeInformation<?> typeInfo = TypeExtractor.createTypeInfo(type);
-
-		if (typeInfo instanceof PojoTypeInfo) {
-			config.registerPojoType(type);
-		} else {
-			config.registerKryoType(type);
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Time characteristic
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Sets the time characteristic for all streams create from this environment, e.g., processing
-	 * time, event time, or ingestion time.
-	 *
-	 * <p>
-	 * If you set the characteristic to IngestionTime of EventTime this will set a default
-	 * watermark update interval of 200 ms. If this is not applicable for your application
-	 * you should change it using {@link ExecutionConfig#setAutoWatermarkInterval(long)}.
-	 * 
-	 * @param characteristic The time characteristic.
-	 */
-	public void setStreamTimeCharacteristic(TimeCharacteristic characteristic) {
-		this.timeCharacteristic = requireNonNull(characteristic);
-		if (characteristic == TimeCharacteristic.ProcessingTime) {
-			getConfig().disableTimestamps();
-			getConfig().setAutoWatermarkInterval(0);
-		} else {
-			getConfig().enableTimestamps();
-			getConfig().setAutoWatermarkInterval(200);
-		}
-	}
-
-	/**
-	 * Gets the time characteristic.
-	 *
-	 * @see #setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)
-	 *
-	 * @return The time characteristic.
-	 */
-	public TimeCharacteristic getStreamTimeCharacteristic() {
-		return timeCharacteristic;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// Data stream creations
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Creates a new data stream that contains a sequence of numbers. This is a parallel source,
-	 * if you manually set the parallelism to {@code 1}
-	 * (using {@link org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator#setParallelism(int)})
-	 * the generated sequence of elements is in order.
-	 *
-	 * @param from
-	 * 		The number to start at (inclusive)
-	 * @param to
-	 * 		The number to stop at (inclusive)
-	 * @return A data stream, containing all number in the [from, to] interval
-	 */
-	public DataStreamSource<Long> generateSequence(long from, long to) {
-		if (from > to) {
-			throw new IllegalArgumentException("Start of sequence must not be greater than the end");
-		}
-		return addSource(new StatefulSequenceSource(from, to), "Sequence Source");
-	}
-
-	/**
-	 * Creates a new data stream that contains the given elements. The elements must all be of the same type, for
-	 * example, all of the {@link String} or {@link Integer}.
-	 * <p>
-	 * The framework will try and determine the exact type from the elements. In case of generic elements, it may be
-	 * necessary to manually supply the type information via {@link #fromCollection(java.util.Collection,
-	 * org.apache.flink.api.common.typeinfo.TypeInformation)}.
-	 * <p>
-	 * Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with a
-	 * degree of parallelism one.
-	 *
-	 * @param data
-	 * 		The array of elements to create the data stream from.
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return The data stream representing the given array of elements
-	 */
-	@SafeVarargs
-	public final <OUT> DataStreamSource<OUT> fromElements(OUT... data) {
-		if (data.length == 0) {
-			throw new IllegalArgumentException("fromElements needs at least one element as argument");
-		}
-
-		TypeInformation<OUT> typeInfo;
-		try {
-			typeInfo = TypeExtractor.getForObject(data[0]);
-		}
-		catch (Exception e) {
-			throw new RuntimeException("Could not create TypeInformation for type " + data[0].getClass().getName()
-				+ "; please specify the TypeInformation manually via "
-				+ "StreamExecutionEnvironment#fromElements(Collection, TypeInformation)");
-		}
-		return fromCollection(Arrays.asList(data), typeInfo);
-	}
-
-	/**
-	 * Creates a data stream from the given non-empty collection. The type of the data stream is that of the
-	 * elements in the collection.
-	 *
-	 * <p>The framework will try and determine the exact type from the collection elements. In case of generic
-	 * elements, it may be necessary to manually supply the type information via
-	 * {@link #fromCollection(java.util.Collection, org.apache.flink.api.common.typeinfo.TypeInformation)}.</p>
-	 *
-	 * <p>Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with a
-	 * parallelism one.</p>
-	 *
-	 * @param data
-	 * 		The collection of elements to create the data stream from.
-	 * @param <OUT>
-	 *     The generic type of the returned data stream.
-	 * @return
-	 *     The data stream representing the given collection
-	 */
-	public <OUT> DataStreamSource<OUT> fromCollection(Collection<OUT> data) {
-		Preconditions.checkNotNull(data, "Collection must not be null");
-		if (data.isEmpty()) {
-			throw new IllegalArgumentException("Collection must not be empty");
-		}
-
-		OUT first = data.iterator().next();
-		if (first == null) {
-			throw new IllegalArgumentException("Collection must not contain null elements");
-		}
-
-		TypeInformation<OUT> typeInfo;
-		try {
-			typeInfo = TypeExtractor.getForObject(first);
-		}
-		catch (Exception e) {
-			throw new RuntimeException("Could not create TypeInformation for type " + first.getClass()
-					+ "; please specify the TypeInformation manually via "
-					+ "StreamExecutionEnvironment#fromElements(Collection, TypeInformation)");
-		}
-		return fromCollection(data, typeInfo);
-	}
-
-	/**
-	 * Creates a data stream from the given non-empty collection.
-	 * 
-	 * <p>Note that this operation will result in a non-parallel data stream source,
-	 * i.e., a data stream source with a parallelism one.</p>
-	 *
-	 * @param data
-	 * 		The collection of elements to create the data stream from
-	 * @param typeInfo
-	 * 		The TypeInformation for the produced data stream
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return The data stream representing the given collection
-	 */
-	public <OUT> DataStreamSource<OUT> fromCollection(Collection<OUT> data, TypeInformation<OUT> typeInfo) {
-		Preconditions.checkNotNull(data, "Collection must not be null");
-		
-		// must not have null elements and mixed elements
-		FromElementsFunction.checkCollection(data, typeInfo.getTypeClass());
-
-		SourceFunction<OUT> function;
-		try {
-			function = new FromElementsFunction<OUT>(typeInfo.createSerializer(getConfig()), data);
-		}
-		catch (IOException e) {
-			throw new RuntimeException(e.getMessage(), e);
-		}
-		return addSource(function, "Collection Source", typeInfo).setParallelism(1);
-	}
-
-	/**
-	 * Creates a data stream from the given iterator.
-	 * 
-	 * <p>Because the iterator will remain unmodified until the actual execution happens,
-	 * the type of data returned by the iterator must be given explicitly in the form of the type
-	 * class (this is due to the fact that the Java compiler erases the generic type information).</p>
-	 * 
-	 * <p>Note that this operation will result in a non-parallel data stream source, i.e.,
-	 * a data stream source with a parallelism of one.</p>
-	 *
-	 * @param data
-	 * 		The iterator of elements to create the data stream from
-	 * @param type
-	 * 		The class of the data produced by the iterator. Must not be a generic class.
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return The data stream representing the elements in the iterator
-	 * @see #fromCollection(java.util.Iterator, org.apache.flink.api.common.typeinfo.TypeInformation)
-	 */
-	public <OUT> DataStreamSource<OUT> fromCollection(Iterator<OUT> data, Class<OUT> type) {
-		return fromCollection(data, TypeExtractor.getForClass(type));
-	}
-
-	/**
-	 * Creates a data stream from the given iterator.
-	 * 
-	 * <p>Because the iterator will remain unmodified until the actual execution happens,
-	 * the type of data returned by the iterator must be given explicitly in the form of the type
-	 * information. This method is useful for cases where the type is generic.
-	 * In that case, the type class (as given in
-	 * {@link #fromCollection(java.util.Iterator, Class)} does not supply all type information.</p>
-	 * 
-	 * <p>Note that this operation will result in a non-parallel data stream source, i.e.,
-	 * a data stream source with a parallelism one.</p>
-	 *
-	 * @param data
-	 * 		The iterator of elements to create the data stream from
-	 * @param typeInfo
-	 * 		The TypeInformation for the produced data stream
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return The data stream representing the elements in the iterator
-	 */
-	public <OUT> DataStreamSource<OUT> fromCollection(Iterator<OUT> data, TypeInformation<OUT> typeInfo) {
-		Preconditions.checkNotNull(data, "The iterator must not be null");
-
-		SourceFunction<OUT> function = new FromIteratorFunction<OUT>(data);
-		return addSource(function, "Collection Source", typeInfo);
-	}
-
-	/**
-	 * Creates a new data stream that contains elements in the iterator. The iterator is splittable, allowing the
-	 * framework to create a parallel data stream source that returns the elements in the iterator.
-	 * 
-	 * <p>Because the iterator will remain unmodified until the actual execution happens, the type of data returned by the
-	 * iterator must be given explicitly in the form of the type class (this is due to the fact that the Java compiler
-	 * erases the generic type information).</p>
-	 *
-	 * @param iterator
-	 * 		The iterator that produces the elements of the data stream
-	 * @param type
-	 * 		The class of the data produced by the iterator. Must not be a generic class.
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return A data stream representing the elements in the iterator
-	 */
-	public <OUT> DataStreamSource<OUT> fromParallelCollection(SplittableIterator<OUT> iterator, Class<OUT> type) {
-		return fromParallelCollection(iterator, TypeExtractor.getForClass(type));
-	}
-
-	/**
-	 * Creates a new data stream that contains elements in the iterator. The iterator is splittable, allowing the
-	 * framework to create a parallel data stream source that returns the elements in the iterator.
-	 * <p>
-	 * Because the iterator will remain unmodified until the actual execution happens, the type of data returned by the
-	 * iterator must be given explicitly in the form of the type information. This method is useful for cases where the
-	 * type is generic. In that case, the type class (as given in {@link #fromParallelCollection(org.apache.flink.util.SplittableIterator,
-	 * Class)} does not supply all type information.
-	 *
-	 * @param iterator
-	 * 		The iterator that produces the elements of the data stream
-	 * @param typeInfo
-	 * 		The TypeInformation for the produced data stream.
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return A data stream representing the elements in the iterator
-	 */
-	public <OUT> DataStreamSource<OUT> fromParallelCollection(SplittableIterator<OUT> iterator, TypeInformation<OUT>
-			typeInfo) {
-		return fromParallelCollection(iterator, typeInfo, "Parallel Collection Source");
-	}
-
-	// private helper for passing different names
-	private <OUT> DataStreamSource<OUT> fromParallelCollection(SplittableIterator<OUT> iterator, TypeInformation<OUT>
-			typeInfo, String operatorName) {
-		return addSource(new FromSplittableIteratorFunction<OUT>(iterator), operatorName).returns(typeInfo);
-	}
-
-	/**
-	 * Creates a data stream that represents the Strings produced by reading the given file line wise. The file will be
-	 * read with the system's default character set.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path").
-	 * @return The data stream that represents the data read from the given file as text lines
-	 */
-	public DataStreamSource<String> readTextFile(String filePath) {
-		Preconditions.checkNotNull(filePath, "The file path may not be null.");
-		TextInputFormat format = new TextInputFormat(new Path(filePath));
-		TypeInformation<String> typeInfo = BasicTypeInfo.STRING_TYPE_INFO;
-
-		return createInput(format, typeInfo, "Read Text File Source");
-	}
-
-	/**
-	 * Creates a data stream that represents the Strings produced by reading the given file line wise. The {@link
-	 * java.nio.charset.Charset} with the given name will be used to read the files.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path")
-	 * @param charsetName
-	 * 		The name of the character set used to read the file
-	 * @return The data stream that represents the data read from the given file as text lines
-	 */
-	public DataStreamSource<String> readTextFile(String filePath, String charsetName) {
-		Preconditions.checkNotNull(filePath, "The file path may not be null.");
-		TextInputFormat format = new TextInputFormat(new Path(filePath));
-		TypeInformation<String> typeInfo = BasicTypeInfo.STRING_TYPE_INFO;
-		format.setCharsetName(charsetName);
-
-		return createInput(format, typeInfo, "Read Text File Source");
-	}
-
-	/**
-	 * Creates a data stream that represents the strings produced by reading the given file line wise. This method is
-	 * similar to {@link #readTextFile(String)}, but it produces a data stream with mutable {@link org.apache.flink.types.StringValue}
-	 * objects,
-	 * rather than Java Strings. StringValues can be used to tune implementations to be less object and garbage
-	 * collection heavy.
-	 * <p/>
-	 * The file will be read with the system's default character set.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path")
-	 * @return A data stream that represents the data read from the given file as text lines
-	 */
-	public DataStreamSource<StringValue> readTextFileWithValue(String filePath) {
-		Preconditions.checkNotNull(filePath, "The file path may not be null.");
-		TextValueInputFormat format = new TextValueInputFormat(new Path(filePath));
-		TypeInformation<StringValue> typeInfo = new ValueTypeInfo<StringValue>(StringValue.class);
-
-		return createInput(format, typeInfo, "Read Text File with Value " +
-				"source");
-	}
-
-	/**
-	 * Creates a data stream that represents the Strings produced by reading the given file line wise. This method is
-	 * similar to {@link #readTextFile(String, String)}, but it produces a data stream with mutable {@link org.apache.flink.types.StringValue}
-	 * objects, rather than Java Strings. StringValues can be used to tune implementations to be less object and
-	 * garbage
-	 * collection heavy.
-	 * <p/>
-	 * The {@link java.nio.charset.Charset} with the given name will be used to read the files.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path")
-	 * @param charsetName
-	 * 		The name of the character set used to read the file
-	 * @param skipInvalidLines
-	 * 		A flag to indicate whether to skip lines that cannot be read with the given character set
-	 * @return A data stream that represents the data read from the given file as text lines
-	 */
-	public DataStreamSource<StringValue> readTextFileWithValue(String filePath, String charsetName, boolean
-			skipInvalidLines) {
-		Preconditions.checkNotNull(filePath, "The file path may not be null.");
-
-		TextValueInputFormat format = new TextValueInputFormat(new Path(filePath));
-		TypeInformation<StringValue> typeInfo = new ValueTypeInfo<StringValue>(StringValue.class);
-		format.setCharsetName(charsetName);
-		format.setSkipInvalidLines(skipInvalidLines);
-		return createInput(format, typeInfo, "Read Text File with Value " +
-				"source");
-	}
-
-	/**
-	 * Reads the given file with the given imput format.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path")
-	 * @param inputFormat
-	 * 		The input format used to create the data stream
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return The data stream that represents the data read from the given file
-	 */
-	public <OUT> DataStreamSource<OUT> readFile(FileInputFormat<OUT> inputFormat, String filePath) {
-		Preconditions.checkNotNull(inputFormat, "InputFormat must not be null.");
-		Preconditions.checkNotNull(filePath, "The file path must not be null.");
-
-		inputFormat.setFilePath(new Path(filePath));
-		try {
-			return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat), "Read File source");
-		} catch (Exception e) {
-			throw new InvalidProgramException("The type returned by the input format could not be automatically " +
-					"determined. " +
-					"Please specify the TypeInformation of the produced type explicitly by using the " +
-					"'createInput(InputFormat, TypeInformation)' method instead.");
-		}
-	}
-
-	/**
-	 * Creates a data stream that represents the primitive type produced by reading the given file line wise.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path")
-	 * @param typeClass
-	 * 		The primitive type class to be read
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return A data stream that represents the data read from the given file as primitive type
-	 */
-	public <OUT> DataStreamSource<OUT> readFileOfPrimitives(String filePath, Class<OUT> typeClass) {
-		Preconditions.checkNotNull(filePath, "The file path may not be null.");
-		PrimitiveInputFormat<OUT> inputFormat = new PrimitiveInputFormat<OUT>(new Path(filePath), typeClass);
-		TypeInformation<OUT> typeInfo = TypeExtractor.getForClass(typeClass);
-
-		return createInput(inputFormat, typeInfo, "Read File of Primitives source");
-	}
-
-	/**
-	 * Creates a data stream that represents the primitive type produced by reading the given file in delimited way.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path")
-	 * @param delimiter
-	 * 		The delimiter of the given file
-	 * @param typeClass
-	 * 		The primitive type class to be read
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return A data stream that represents the data read from the given file as primitive type.
-	 */
-	public <OUT> DataStreamSource<OUT> readFileOfPrimitives(String filePath, String delimiter, Class<OUT> typeClass) {
-		Preconditions.checkNotNull(filePath, "The file path may not be null.");
-		PrimitiveInputFormat<OUT> inputFormat = new PrimitiveInputFormat<OUT>(new Path(filePath), delimiter,
-				typeClass);
-		TypeInformation<OUT> typeInfo = TypeExtractor.getForClass(typeClass);
-
-		return createInput(inputFormat, typeInfo, "Read File of Primitives source");
-	}
-
-	/**
-	 * Creates a data stream that contains the contents of file created while system watches the given path. The file
-	 * will be read with the system's default character set.
-	 *
-	 * @param filePath
-	 * 		The path of the file, as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path/")
-	 * @param intervalMillis
-	 * 		The interval of file watching in milliseconds
-	 * @param watchType
-	 * 		The watch type of file stream. When watchType is {@link org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType#ONLY_NEW_FILES}, the system processes
-	 * 		only
-	 * 		new files. {@link org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType#REPROCESS_WITH_APPENDED} means that the system re-processes all contents of
-	 * 		appended file. {@link org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType#PROCESS_ONLY_APPENDED} means that the system processes only appended
-	 * 		contents
-	 * 		of files.
-	 * @return The DataStream containing the given directory.
-	 */
-	public DataStream<String> readFileStream(String filePath, long intervalMillis,
-											WatchType watchType) {
-		DataStream<Tuple3<String, Long, Long>> source = addSource(new FileMonitoringFunction(
-				filePath, intervalMillis, watchType), "Read File Stream source");
-
-		return source.flatMap(new FileReadFunction());
-	}
-
-	/**
-	 * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are
-	 * decoded by the system's default character set. On the termination of the socket server connection retries can be
-	 * initiated.
-	 * <p/>
-	 * Let us note that the socket itself does not report on abort and as a consequence retries are only initiated when
-	 * the socket was gracefully terminated.
-	 *
-	 * @param hostname
-	 * 		The host name which a server socket binds
-	 * @param port
-	 * 		The port number which a server socket binds. A port number of 0 means that the port number is automatically
-	 * 		allocated.
-	 * @param delimiter
-	 * 		A character which splits received strings into records
-	 * @param maxRetry
-	 * 		The maximal retry interval in seconds while the program waits for a socket that is temporarily down.
-	 * 		Reconnection is initiated every second. A number of 0 means that the reader is immediately terminated,
-	 * 		while
-	 * 		a	negative value ensures retrying forever.
-	 * @return A data stream containing the strings received from the socket
-	 */
-	public DataStreamSource<String> socketTextStream(String hostname, int port, char delimiter, long maxRetry) {
-		return addSource(new SocketTextStreamFunction(hostname, port, delimiter, maxRetry),
-				"Socket Stream");
-	}
-
-	/**
-	 * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are
-	 * decoded by the system's default character set. The reader is terminated immediately when the socket is down.
-	 *
-	 * @param hostname
-	 * 		The host name which a server socket binds
-	 * @param port
-	 * 		The port number which a server socket binds. A port number of 0 means that the port number is automatically
-	 * 		allocated.
-	 * @param delimiter
-	 * 		A character which splits received strings into records
-	 * @return A data stream containing the strings received from the socket
-	 */
-	public DataStreamSource<String> socketTextStream(String hostname, int port, char delimiter) {
-		return socketTextStream(hostname, port, delimiter, 0);
-	}
-
-	/**
-	 * Creates a new data stream that contains the strings received infinitely from a socket. Received strings are
-	 * decoded by the system's default character set, using'\n' as delimiter. The reader is terminated immediately when
-	 * the socket is down.
-	 *
-	 * @param hostname
-	 * 		The host name which a server socket binds
-	 * @param port
-	 * 		The port number which a server socket binds. A port number of 0 means that the port number is automatically
-	 * 		allocated.
-	 * @return A data stream containing the strings received from the socket
-	 */
-	public DataStreamSource<String> socketTextStream(String hostname, int port) {
-		return socketTextStream(hostname, port, '\n');
-	}
-
-	/**
-	 * Generic method to create an input data stream with {@link org.apache.flink.api.common.io.InputFormat}.
-	 * <p/>
-	 * Since all data streams need specific information about their types, this method needs to determine the type of
-	 * the data produced by the input format. It will attempt to determine the data type by reflection, unless the
-	 * input
-	 * format implements the {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable} interface. In the latter
-	 * case, this method will invoke the {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable#getProducedType()}
-	 * method to determine data type produced by the input format.
-	 *
-	 * @param inputFormat
-	 * 		The input format used to create the data stream
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return The data stream that represents the data created by the input format
-	 */
-	public <OUT> DataStreamSource<OUT> createInput(InputFormat<OUT, ?> inputFormat) {
-		return createInput(inputFormat, TypeExtractor.getInputFormatTypes(inputFormat), "Custom File source");
-	}
-
-	/**
-	 * Generic method to create an input data stream with {@link org.apache.flink.api.common.io.InputFormat}.
-	 * <p>
-	 * The data stream is typed to the given TypeInformation. This method is intended for input formats where the
-	 * return
-	 * type cannot be determined by reflection analysis, and that do not implement the
-	 * {@link org.apache.flink.api.java.typeutils.ResultTypeQueryable} interface.
-	 *
-	 * @param inputFormat
-	 * 		The input format used to create the data stream
-	 * @param <OUT>
-	 * 		The type of the returned data stream
-	 * @return The data stream that represents the data created by the input format
-	 */
-	public <OUT> DataStreamSource<OUT> createInput(InputFormat<OUT, ?> inputFormat, TypeInformation<OUT> typeInfo) {
-		return createInput(inputFormat, typeInfo, "Custom File source");
-	}
-
-	// private helper for passing different names
-	private <OUT> DataStreamSource<OUT> createInput(InputFormat<OUT, ?> inputFormat,
-			TypeInformation<OUT> typeInfo, String sourceName) {
-		FileSourceFunction<OUT> function = new FileSourceFunction<OUT>(inputFormat, typeInfo);
-		return addSource(function, sourceName).returns(typeInfo);
-	}
-
-	/**
-	 * Adds a Data Source to the streaming topology.
-	 *
-	 * <p>
-	 * By default sources have a parallelism of 1. To enable parallel execution, the user defined source should
-	 * implement {@link org.apache.flink.streaming.api.functions.source.ParallelSourceFunction} or extend {@link
-	 * org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction}. In these cases the resulting source
-	 * will have the parallelism of the environment. To change this afterwards call {@link
-	 * org.apache.flink.streaming.api.datastream.DataStreamSource#setParallelism(int)}
-	 *
-	 * @param function
-	 * 		the user defined function
-	 * @param <OUT>
-	 * 		type of the returned stream
-	 * @return the data stream constructed
-	 */
-	public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> function) {
-		return addSource(function, "Custom Source");
-	}
-
-	/**
-	 * Ads a data source with a custom type information thus opening a
-	 * {@link DataStream}. Only in very special cases does the user need to
-	 * support type information. Otherwise use
-	 * {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)}
-	 *
-	 * @param function
-	 * 		the user defined function
-	 * @param sourceName
-	 * 		Name of the data source
-	 * @param <OUT>
-	 * 		type of the returned stream
-	 * @return the data stream constructed
-	 */
-	public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> function, String sourceName) {
-		return addSource(function, sourceName, null);
-	}
-
-	/**
-	 * Ads a data source with a custom type information thus opening a
-	 * {@link DataStream}. Only in very special cases does the user need to
-	 * support type information. Otherwise use
-	 * {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)}
-	 *
-	 * @param function
-	 * 		the user defined function
-	 * @param <OUT>
-	 * 		type of the returned stream
-	 * @param typeInfo
-	 * 		the user defined type information for the stream
-	 * @return the data stream constructed
-	 */
-	public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> function, TypeInformation<OUT> typeInfo) {
-		return addSource(function, "Custom Source", typeInfo);
-	}
-
-	/**
-	 * Ads a data source with a custom type information thus opening a
-	 * {@link DataStream}. Only in very special cases does the user need to
-	 * support type information. Otherwise use
-	 * {@link #addSource(org.apache.flink.streaming.api.functions.source.SourceFunction)}
-	 *
-	 * @param function
-	 * 		the user defined function
-	 * @param sourceName
-	 * 		Name of the data source
-	 * @param <OUT>
-	 * 		type of the returned stream
-	 * @param typeInfo
-	 * 		the user defined type information for the stream
-	 * @return the data stream constructed
-	 */
-	@SuppressWarnings("unchecked")
-	public <OUT> DataStreamSource<OUT> addSource(SourceFunction<OUT> function, String sourceName, TypeInformation<OUT> typeInfo) {
-
-		if(typeInfo == null) {
-			if (function instanceof ResultTypeQueryable) {
-				typeInfo = ((ResultTypeQueryable<OUT>) function).getProducedType();
-			} else {
-				try {
-					typeInfo = TypeExtractor.createTypeInfo(
-							SourceFunction.class,
-							function.getClass(), 0, null, null);
-				} catch (final InvalidTypesException e) {
-					typeInfo = (TypeInformation<OUT>) new MissingTypeInfo(sourceName, e);
-				}
-			}
-		}
-
-		boolean isParallel = function instanceof ParallelSourceFunction;
-
-		clean(function);
-		StreamSource<OUT> sourceOperator = new StreamSource<OUT>(function);
-
-		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);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Factory methods for ExecutionEnvironments
-	// --------------------------------------------------------------------------------------------
-
-	/**
-	 * Creates an execution environment that represents the context in which the
-	 * program is currently executed. If the program is invoked standalone, this
-	 * method returns a local execution environment, as returned by
-	 * {@link #createLocalEnvironment()}.
-	 *
-	 * @return The execution environment of the context in which the program is
-	 * executed.
-	 */
-	public static StreamExecutionEnvironment getExecutionEnvironment() {
-		if (contextEnvironmentFactory != null) {
-			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;
-			return createContextEnvironment(ctx.getClient(), ctx.getJars(), ctx.getClasspaths(),
-					ctx.getParallelism(), ctx.isWait());
-		} else if (env instanceof OptimizerPlanEnvironment | env instanceof PreviewPlanEnvironment) {
-			return new StreamPlanEnvironment(env);
-		} else {
-			return createLocalEnvironment();
-		}
-	}
-
-	private static StreamExecutionEnvironment createContextEnvironment(
-			Client client, List<URL> jars, List<URL> classpaths, int parallelism, boolean wait)
-	{
-		return new StreamContextEnvironment(client, jars, classpaths, parallelism, wait);
-	}
-
-	/**
-	 * Creates a {@link LocalStreamEnvironment}. The local execution environment
-	 * will run the program in a multi-threaded fashion in the same JVM as the
-	 * environment was created in. The default parallelism of the local
-	 * environment is the number of hardware contexts (CPU cores / threads),
-	 * unless it was specified differently by {@link #setParallelism(int)}.
-	 *
-	 * @return A local execution environment.
-	 */
-	public static LocalStreamEnvironment createLocalEnvironment() {
-		return createLocalEnvironment(defaultLocalParallelism);
-	}
-
-	/**
-	 * Creates a {@link LocalStreamEnvironment}. The local execution environment
-	 * will run the program in a multi-threaded fashion in the same JVM as the
-	 * environment was created in. It will use the parallelism specified in the
-	 * parameter.
-	 *
-	 * @param parallelism
-	 * 		The parallelism for the local environment.
-	 * @return A local execution environment with the specified parallelism.
-	 */
-	public static LocalStreamEnvironment createLocalEnvironment(int parallelism) {
-		LocalStreamEnvironment env = new LocalStreamEnvironment();
-		env.setParallelism(parallelism);
-		return env;
-	}
-
-	/**
-	 * Creates a {@link LocalStreamEnvironment}. The local execution environment
-	 * will run the program in a multi-threaded fashion in the same JVM as the
-	 * environment was created in. It will use the parallelism specified in the
-	 * parameter.
-	 *
-	 * @param parallelism
-	 * 		The parallelism for the local environment.
-	 * 	@param configuration
-	 * 		Pass a custom configuration into the cluster
-	 * @return A local execution environment with the specified parallelism.
-	 */
-	public static LocalStreamEnvironment createLocalEnvironment(int parallelism, Configuration configuration) {
-		LocalStreamEnvironment currentEnvironment = new LocalStreamEnvironment(configuration);
-		currentEnvironment.setParallelism(parallelism);
-		return currentEnvironment;
-	}
-
-	/**
-	 * 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 no parallelism, unless the parallelism is set
-	 * explicitly via {@link #setParallelism}.
-	 *
-	 * @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 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 StreamExecutionEnvironment createRemoteEnvironment(
-			String host, int port, String... jarFiles) {
-		return new RemoteStreamEnvironment(host, port, jarFiles);
-	}
-
-	/**
-	 * 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.
-	 *
-	 * @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 parallelism
-	 * 		The parallelism to use during the execution.
-	 * @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 StreamExecutionEnvironment createRemoteEnvironment(
-			String host, int port, int parallelism, String... jarFiles)
-	{
-		RemoteStreamEnvironment env = new RemoteStreamEnvironment(host, port, jarFiles);
-		env.setParallelism(parallelism);
-		return env;
-	}
-
-	/**
-	 * 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.
-	 *
-	 * @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 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
-	// --------------------------------------------------------------------------------------------
-
-	protected static void initializeContextEnvironment(StreamExecutionEnvironmentFactory ctx) {
-		contextEnvironmentFactory = ctx;
-	}
-	
-	protected static void resetContextEnvironment() {
-		contextEnvironmentFactory = null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentFactory.java
deleted file mode 100644
index 0e24bac..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentFactory.java
+++ /dev/null
@@ -1,31 +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.environment;
-
-/**
- * Factory class for stream execution environments.
- */
-public interface StreamExecutionEnvironmentFactory {
-
-	/**
-	 * Creates a StreamExecutionEnvironment from this factory.
-	 *
-	 * @return A StreamExecutionEnvironment.
-	 */
-	StreamExecutionEnvironment createExecutionEnvironment();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.java
deleted file mode 100644
index 61b9a2f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamPlanEnvironment.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.environment;
-
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.client.program.OptimizerPlanEnvironment;
-import org.apache.flink.client.program.PreviewPlanEnvironment;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-
-public class StreamPlanEnvironment extends StreamExecutionEnvironment {
-
-	private ExecutionEnvironment env;
-
-	protected StreamPlanEnvironment(ExecutionEnvironment env) {
-		super();
-		this.env = env;
-
-		int parallelism = env.getParallelism();
-		if (parallelism > 0) {
-			setParallelism(parallelism);
-		} else {
-			// determine parallelism
-			setParallelism(GlobalConfiguration.getInteger(
-					ConfigConstants.DEFAULT_PARALLELISM_KEY,
-					ConfigConstants.DEFAULT_PARALLELISM));
-		}
-	}
-
-	@Override
-	public JobExecutionResult execute() throws Exception {
-		return execute("");
-	}
-
-	@Override
-	public JobExecutionResult execute(String jobName) throws Exception {
-
-		StreamGraph streamGraph = getStreamGraph();
-		streamGraph.setJobName(jobName);
-
-		transformations.clear();
-
-		if (env instanceof OptimizerPlanEnvironment) {
-			((OptimizerPlanEnvironment) env).setPlan(streamGraph);
-		} else if (env instanceof PreviewPlanEnvironment) {
-			((PreviewPlanEnvironment) env).setPreview(streamGraph.getStreamingPlanAsJSON());
-		}
-
-		throw new OptimizerPlanEnvironment.ProgramAbortException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/AscendingTimestampExtractor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/AscendingTimestampExtractor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/AscendingTimestampExtractor.java
deleted file mode 100644
index 85433f5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/AscendingTimestampExtractor.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.functions;
-
-/**
- * Interface for user functions that extract timestamps from elements. The extracting timestamps
- * must be monotonically increasing.
- *
- * @param <T> The type of the elements that this function can extract timestamps from
- */
-public abstract class AscendingTimestampExtractor<T> implements TimestampExtractor<T> {
-
-	long currentTimestamp = 0;
-
-	/**
-	 * Extracts a timestamp from an element. The timestamp must be monotonically increasing.
-	 *
-	 * @param element The element that the timestamp is extracted from.
-	 * @param currentTimestamp The current internal timestamp of the element.
-	 * @return The new timestamp.
-	 */
-	public abstract long extractAscendingTimestamp(T element, long currentTimestamp);
-
-	@Override
-	public final long extractTimestamp(T element, long currentTimestamp) {
-		long newTimestamp = extractAscendingTimestamp(element, currentTimestamp);
-		if (newTimestamp < this.currentTimestamp) {
-			throw new RuntimeException("Timestamp is lower than previously extracted timestamp. " +
-					"You should implement a custom TimestampExtractor.");
-		}
-		this.currentTimestamp = newTimestamp;
-		return this.currentTimestamp;
-	}
-
-	@Override
-	public final long extractWatermark(T element, long currentTimestamp) {
-		return Long.MIN_VALUE;
-	}
-
-	@Override
-	public final long getCurrentWatermark() {
-		return currentTimestamp - 1;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/TimestampExtractor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/TimestampExtractor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/TimestampExtractor.java
deleted file mode 100644
index 7fd7b63..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/TimestampExtractor.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.functions;
-
-import org.apache.flink.api.common.functions.Function;
-
-/**
- * Interface for user functions that extract timestamps from elements.
- *
- * <p>
- * The extractor must also keep track of the current watermark. The system will periodically
- * retrieve this watermark using {@link #getCurrentWatermark()} and submit it throughout the topology.
- *
- * <p>
- * Note: If you know that timestamps are monotonically increasing you can use
- * {@link org.apache.flink.streaming.api.functions.AscendingTimestampExtractor}. This will
- * keep track of watermarks.
- *
- * @see org.apache.flink.streaming.api.watermark.Watermark
- *
- * @param <T> The type of the elements that this function can extract timestamps from
- */
-public interface TimestampExtractor<T> extends Function {
-
-	/**
-	 * Extracts a timestamp from an element.
-	 *
-	 * @param element The element that the timestamp is extracted from.
-	 * @param currentTimestamp The current internal timestamp of the element.
-	 * @return The new timestamp.
-	 */
-	long extractTimestamp(T element, long currentTimestamp);
-
-	/**
-	 * Asks the extractor if it wants to emit a watermark now that it has seen the given element.
-	 * This is called right after {@link #extractTimestamp}. With the same element. The method
-	 * can return {@code Long.MIN_VALUE} to indicate that no watermark should be emitted, a value of 0 or
-	 * greater will be emitted as a watermark if it is higher than the last-emitted watermark.
-	 *
-	 * @param element The element that we last saw.
-	 * @param currentTimestamp The current timestamp of the element that we last saw.
-	 * @return {@code Long.MIN_VALUE} if no watermark should be emitted, positive value for
-	 *          emitting this value as a watermark.
-	 */
-	long extractWatermark(T element, long currentTimestamp);
-
-	/**
-	 * Returns the current watermark. This is periodically called by the system to determine
-	 * the current watermark and forward it.
-	 *
-	 * @see org.apache.flink.streaming.api.watermark.Watermark
-	 */
-	long getCurrentWatermark();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/AggregationFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/AggregationFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/AggregationFunction.java
deleted file mode 100644
index 23cca90..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/AggregationFunction.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.aggregation;
-
-import org.apache.flink.api.common.functions.RichReduceFunction;
-
-public abstract class AggregationFunction<T> extends RichReduceFunction<T> {
-	private static final long serialVersionUID = 1L;
-
-	public int position;
-
-	public AggregationFunction(int pos) {
-		this.position = pos;
-	}
-
-	public static enum AggregationType {
-		SUM, MIN, MAX, MINBY, MAXBY,
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
deleted file mode 100644
index e5501a0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.aggregation;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.util.FieldAccessor;
-
-public class ComparableAggregator<T> extends AggregationFunction<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	public Comparator comparator;
-	public boolean byAggregate;
-	public boolean first;
-	FieldAccessor<T, Object> fieldAccessor;
-	
-	private ComparableAggregator(int pos, AggregationType aggregationType, boolean first) {
-		super(pos);
-		this.comparator = Comparator.getForAggregation(aggregationType);
-		this.byAggregate = (aggregationType == AggregationType.MAXBY) || (aggregationType == AggregationType.MINBY);
-		this.first = first;
-	}
-
-	public ComparableAggregator(int positionToAggregate, TypeInformation<T> typeInfo, AggregationType aggregationType
-			, ExecutionConfig config) {
-		this(positionToAggregate, typeInfo, aggregationType, false, config);
-	}
-
-	public ComparableAggregator(int positionToAggregate, TypeInformation<T> typeInfo, AggregationType aggregationType,
-								boolean first, ExecutionConfig config) {
-		this(positionToAggregate, aggregationType, first);
-		this.fieldAccessor = FieldAccessor.create(positionToAggregate, typeInfo, config);
-		this.first = first;
-	}
-
-	public ComparableAggregator(String field,
-			TypeInformation<T> typeInfo, AggregationType aggregationType, boolean first, ExecutionConfig config) {
-		this(0, aggregationType, first);
-		this.fieldAccessor = FieldAccessor.create(field, typeInfo, config);
-		this.first = first;
-	}
-
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public T reduce(T value1, T value2) throws Exception {
-		Comparable<Object> o1 = (Comparable<Object>) fieldAccessor.get(value1);
-		Object o2 = fieldAccessor.get(value2);
-
-		int c = comparator.isExtremal(o1, o2);
-
-		if (byAggregate) {
-			if (c == 1) {
-				return value1;
-			}
-			if (first) {
-				if (c == 0) {
-					return value1;
-				}
-			}
-
-			return value2;
-
-		} else {
-			if (c == 0) {
-				value1 = fieldAccessor.set(value1, o2);
-			}
-			return value1;
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/Comparator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/Comparator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/Comparator.java
deleted file mode 100644
index f85a2e1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/Comparator.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.aggregation;
-
-import java.io.Serializable;
-
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType;
-
-public abstract class Comparator implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	public abstract <R> int isExtremal(Comparable<R> o1, R o2);
-
-	public static Comparator getForAggregation(AggregationType type) {
-		switch (type) {
-		case MAX:
-			return new MaxComparator();
-		case MIN:
-			return new MinComparator();
-		case MINBY:
-			return new MinByComparator();
-		case MAXBY:
-			return new MaxByComparator();
-		default:
-			throw new IllegalArgumentException("Unsupported aggregation type.");
-		}
-	}
-
-	private static class MaxComparator extends Comparator {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public <R> int isExtremal(Comparable<R> o1, R o2) {
-			return o1.compareTo(o2) > 0 ? 1 : 0;
-		}
-
-	}
-
-	private static class MaxByComparator extends Comparator {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public <R> int isExtremal(Comparable<R> o1, R o2) {
-			int c = o1.compareTo(o2);
-			if (c > 0) {
-				return 1;
-			}
-			if (c == 0) {
-				return 0;
-			} else {
-				return -1;
-			}
-		}
-
-	}
-
-	private static class MinByComparator extends Comparator {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public <R> int isExtremal(Comparable<R> o1, R o2) {
-			int c = o1.compareTo(o2);
-			if (c < 0) {
-				return 1;
-			}
-			if (c == 0) {
-				return 0;
-			} else {
-				return -1;
-			}
-		}
-
-	}
-
-	private static class MinComparator extends Comparator {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public <R> int isExtremal(Comparable<R> o1, R o2) {
-			return o1.compareTo(o2) < 0 ? 1 : 0;
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
deleted file mode 100644
index b045233..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.aggregation;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.util.FieldAccessor;
-
-public class SumAggregator<T> extends AggregationFunction<T> {
-
-	private static final long serialVersionUID = 1L;
-
-	FieldAccessor<T, Object> fieldAccessor;
-	SumFunction adder;
-
-	public SumAggregator(int pos, TypeInformation<T> typeInfo, ExecutionConfig config) {
-		super(pos);
-		fieldAccessor = FieldAccessor.create(pos, typeInfo, config);
-		adder = SumFunction.getForClass(fieldAccessor.getFieldType().getTypeClass());
-	}
-
-	public SumAggregator(String field, TypeInformation<T> typeInfo, ExecutionConfig config) {
-		super(0);
-		fieldAccessor = FieldAccessor.create(field, typeInfo, config);
-		adder = SumFunction.getForClass(fieldAccessor.getFieldType().getTypeClass());
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public T reduce(T value1, T value2) throws Exception {
-		return fieldAccessor.set(value1, adder.add(fieldAccessor.get(value1), fieldAccessor.get(value2)));
-	}
-}


[43/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
deleted file mode 100644
index 4074a1d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
+++ /dev/null
@@ -1,331 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.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.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
-import org.apache.flink.streaming.api.operators.co.CoStreamMap;
-import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
-
-/**
- * {@code ConnectedStreams} represents two connected streams of (possible) different data types. It
- * can be used to apply transformations such as {@link CoMapFunction} on two
- * {@link DataStream DataStreams}
- * 
- * @param <IN1> Type of the first input data steam.
- * @param <IN2> Type of the second input data stream.
- */
-public class ConnectedStreams<IN1, IN2> {
-
-	protected StreamExecutionEnvironment environment;
-	protected DataStream<IN1> inputStream1;
-	protected DataStream<IN2> inputStream2;
-
-	protected ConnectedStreams(StreamExecutionEnvironment env,
-			DataStream<IN1> input1,
-			DataStream<IN2> input2) {
-		this.environment = env;
-		if (input1 != null) {
-			this.inputStream1 = input1;
-		}
-		if (input2 != null) {
-			this.inputStream2 = input2;
-		}
-	}
-
-	public StreamExecutionEnvironment getExecutionEnvironment() {
-		return environment;
-	}
-
-	/**
-	 * Returns the first {@link DataStream}.
-	 *
-	 * @return The first DataStream.
-	 */
-	public DataStream<IN1> getFirstInput() {
-		return inputStream1;
-	}
-
-	/**
-	 * Returns the second {@link DataStream}.
-	 *
-	 * @return The second DataStream.
-	 */
-	public DataStream<IN2> getSecondInput() {
-		return inputStream2;
-	}
-
-	/**
-	 * Gets the type of the first input
-	 *
-	 * @return The type of the first input
-	 */
-	public TypeInformation<IN1> getType1() {
-		return inputStream1.getType();
-	}
-
-	/**
-	 * Gets the type of the second input
-	 *
-	 * @return The type of the second input
-	 */
-	public TypeInformation<IN2> getType2() {
-		return inputStream2.getType();
-	}
-
-	/**
-	 * KeyBy operation for connected data stream. Assigns keys to the elements of
-	 * input1 and input2 according to keyPosition1 and keyPosition2.
-	 *
-	 * @param keyPosition1
-	 *            The field used to compute the hashcode of the elements in the
-	 *            first input stream.
-	 * @param keyPosition2
-	 *            The field used to compute the hashcode of the elements in the
-	 *            second input stream.
-	 * @return The grouped {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> keyBy(int keyPosition1, int keyPosition2) {
-		return new ConnectedStreams<>(this.environment, inputStream1.keyBy(keyPosition1),
-				inputStream2.keyBy(keyPosition2));
-	}
-
-	/**
-	 * KeyBy operation for connected data stream. Assigns keys to the elements of
-	 * input1 and input2 according to keyPositions1 and keyPositions2.
-	 *
-	 * @param keyPositions1
-	 *            The fields used to group the first input stream.
-	 * @param keyPositions2
-	 *            The fields used to group the second input stream.
-	 * @return The grouped {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> keyBy(int[] keyPositions1, int[] keyPositions2) {
-		return new ConnectedStreams<>(environment, inputStream1.keyBy(keyPositions1),
-				inputStream2.keyBy(keyPositions2));
-	}
-
-	/**
-	 * KeyBy operation for connected data stream using key expressions. Assigns keys to
-	 * the elements of input1 and input2 according to field1 and field2. A field
-	 * expression is either the name of a public field or a getter method with
-	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
-	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field1
-	 *            The grouping expression for the first input
-	 * @param field2
-	 *            The grouping expression for the second input
-	 * @return The grouped {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> keyBy(String field1, String field2) {
-		return new ConnectedStreams<>(environment, inputStream1.keyBy(field1),
-				inputStream2.keyBy(field2));
-	}
-
-	/**
-	 * KeyBy operation for connected data stream using key expressions.
-	 * the elements of input1 and input2 according to fields1 and fields2. A
-	 * field expression is either the name of a public field or a getter method
-	 * with parentheses of the {@link DataStream}S underlying type. A dot can be
-	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-	 * .
-	 *
-	 * @param fields1
-	 *            The grouping expressions for the first input
-	 * @param fields2
-	 *            The grouping expressions for the second input
-	 * @return The grouped {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> keyBy(String[] fields1, String[] fields2) {
-		return new ConnectedStreams<>(environment, inputStream1.keyBy(fields1),
-				inputStream2.keyBy(fields2));
-	}
-
-	/**
-	 * KeyBy operation for connected data stream. Assigns keys to the elements of
-	 * input1 and input2 using keySelector1 and keySelector2.
-	 *
-	 * @param keySelector1
-	 *            The {@link KeySelector} used for grouping the first input
-	 * @param keySelector2
-	 *            The {@link KeySelector} used for grouping the second input
-	 * @return The partitioned {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> keyBy(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
-		return new ConnectedStreams<>(environment, inputStream1.keyBy(keySelector1),
-				inputStream2.keyBy(keySelector2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream. Partitions the elements of
-	 * input1 and input2 according to keyPosition1 and keyPosition2.
-	 *
-	 * @param keyPosition1
-	 *            The field used to compute the hashcode of the elements in the
-	 *            first input stream.
-	 * @param keyPosition2
-	 *            The field used to compute the hashcode of the elements in the
-	 *            second input stream.
-	 * @return The partitioned {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> partitionByHash(int keyPosition1, int keyPosition2) {
-		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keyPosition1),
-				inputStream2.partitionByHash(keyPosition2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream. Partitions the elements of
-	 * input1 and input2 according to keyPositions1 and keyPositions2.
-	 *
-	 * @param keyPositions1
-	 *            The fields used to group the first input stream.
-	 * @param keyPositions2
-	 *            The fields used to group the second input stream.
-	 * @return The partitioned {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> partitionByHash(int[] keyPositions1, int[] keyPositions2) {
-		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keyPositions1),
-				inputStream2.partitionByHash(keyPositions2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream using key expressions. Partitions
-	 * the elements of input1 and input2 according to field1 and field2. A
-	 * field expression is either the name of a public field or a getter method
-	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
-	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-	 *
-	 * @param field1
-	 *            The partitioning expressions for the first input
-	 * @param field2
-	 *            The partitioning expressions for the second input
-	 * @return The partitioned {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> partitionByHash(String field1, String field2) {
-		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(field1),
-				inputStream2.partitionByHash(field2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream using key expressions. Partitions
-	 * the elements of input1 and input2 according to fields1 and fields2. A
-	 * field expression is either the name of a public field or a getter method
-	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
-	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-	 *
-	 * @param fields1
-	 *            The partitioning expressions for the first input
-	 * @param fields2
-	 *            The partitioning expressions for the second input
-	 * @return The partitioned {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> partitionByHash(String[] fields1, String[] fields2) {
-		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(fields1),
-				inputStream2.partitionByHash(fields2));
-	}
-
-	/**
-	 * PartitionBy operation for connected data stream. Partitions the elements of
-	 * input1 and input2 using keySelector1 and keySelector2.
-	 *
-	 * @param keySelector1
-	 *            The {@link KeySelector} used for partitioning the first input
-	 * @param keySelector2
-	 *            The {@link KeySelector} used for partitioning the second input
-	 * @return @return The partitioned {@link ConnectedStreams}
-	 */
-	public ConnectedStreams<IN1, IN2> partitionByHash(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
-		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keySelector1),
-				inputStream2.partitionByHash(keySelector2));
-	}
-
-	/**
-	 * Applies a CoMap transformation on a {@link ConnectedStreams} and maps
-	 * the output to a common type. The transformation calls a
-	 * {@link CoMapFunction#map1} for each element of the first input and
-	 * {@link CoMapFunction#map2} for each element of the second input. Each
-	 * CoMapFunction call returns exactly one element.
-	 * 
-	 * @param coMapper The CoMapFunction used to jointly transform the two input DataStreams
-	 * @return The transformed {@link DataStream}
-	 */
-	public <OUT> SingleOutputStreamOperator<OUT, ?> map(CoMapFunction<IN1, IN2, OUT> coMapper) {
-
-		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coMapper,
-				CoMapFunction.class, false, true, getType1(), getType2(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Co-Map", outTypeInfo, new CoStreamMap<>(inputStream1.clean(coMapper)));
-
-	}
-
-	/**
-	 * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
-	 * maps the output to a common type. The transformation calls a
-	 * {@link CoFlatMapFunction#flatMap1} for each element of the first input
-	 * and {@link CoFlatMapFunction#flatMap2} for each element of the second
-	 * input. Each CoFlatMapFunction call returns any number of elements
-	 * including none.
-	 * 
-	 * @param coFlatMapper
-	 *            The CoFlatMapFunction used to jointly transform the two input
-	 *            DataStreams
-	 * @return The transformed {@link DataStream}
-	 */
-	public <OUT> SingleOutputStreamOperator<OUT, ?> flatMap(
-			CoFlatMapFunction<IN1, IN2, OUT> coFlatMapper) {
-
-		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coFlatMapper,
-				CoFlatMapFunction.class, false, true, getType1(), getType2(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Co-Flat Map", outTypeInfo, new CoStreamFlatMap<>(inputStream1.clean(coFlatMapper)));
-	}
-
-	public <OUT> SingleOutputStreamOperator<OUT, ?> transform(String functionName,
-			TypeInformation<OUT> outTypeInfo,
-			TwoInputStreamOperator<IN1, IN2, OUT> operator) {
-
-		// read the output type of the input Transforms to coax out errors about MissingTypeInfo
-		inputStream1.getType();
-		inputStream2.getType();
-
-		TwoInputTransformation<IN1, IN2, OUT> transform = new TwoInputTransformation<>(
-				inputStream1.getTransformation(),
-				inputStream2.getTransformation(),
-				functionName,
-				operator,
-				outTypeInfo,
-				environment.getParallelism());
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(environment, transform);
-
-		getExecutionEnvironment().addOperator(transform);
-
-		return returnStream;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 176a07f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ /dev/null
@@ -1,1077 +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 java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-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.io.CsvOutputFormat;
-import org.apache.flink.api.java.io.TextOutputFormat;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.core.fs.FileSystem.WriteMode;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.sink.FileSinkFunctionByMillis;
-import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.sink.SocketClientSink;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamFilter;
-import org.apache.flink.streaming.api.operators.StreamFlatMap;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.api.operators.StreamSink;
-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.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.windowing.evictors.CountEvictor;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.streaming.runtime.operators.ExtractTimestampsOperator;
-import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper;
-import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-import com.google.common.base.Preconditions;
-
-/**
- * A DataStream represents a stream of elements of the same type. A DataStream
- * can be transformed into another DataStream by applying a transformation as
- * for example:
- * <ul>
- * <li>{@link DataStream#map},
- * <li>{@link DataStream#filter}, or
- * </ul>
- * 
- * @param <T> The type of the elements in this Stream
- */
-public class DataStream<T> {
-
-	protected final StreamExecutionEnvironment environment;
-
-	protected final StreamTransformation<T> transformation;
-
-	/**
-	 * Create a new {@link DataStream} in the given execution environment with
-	 * partitioning set to forward by default.
-	 *
-	 * @param environment The StreamExecutionEnvironment
-	 */
-	public DataStream(StreamExecutionEnvironment environment, StreamTransformation<T> transformation) {
-		this.environment = Preconditions.checkNotNull(environment, "Execution Environment must not be null.");
-		this.transformation = Preconditions.checkNotNull(transformation, "Stream Transformation must not be null.");
-	}
-
-	/**
-	 * Returns the ID of the {@link DataStream} in the current {@link StreamExecutionEnvironment}.
-	 * 
-	 * @return ID of the DataStream
-	 */
-	public Integer getId() {
-		return transformation.getId();
-	}
-
-	/**
-	 * Gets the parallelism for this operator.
-	 * 
-	 * @return The parallelism set for this operator.
-	 */
-	public int getParallelism() {
-		return transformation.getParallelism();
-	}
-
-	/**
-	 * Gets the type of the stream.
-	 * 
-	 * @return The type of the datastream.
-	 */
-	public TypeInformation<T> getType() {
-		return transformation.getOutputType();
-	}
-
-	/**
-	 * Invokes the {@link org.apache.flink.api.java.ClosureCleaner}
-	 * on the given function if closure cleaning is enabled in the {@link ExecutionConfig}.
-	 *
-	 * @return The cleaned Function
-	 */
-	protected <F> F clean(F f) {
-		return getExecutionEnvironment().clean(f);
-	}
-
-	/**
-	 * Returns the {@link StreamExecutionEnvironment} that was used to create this
-	 * {@link DataStream}
-	 *
-	 * @return The Execution Environment
-	 */
-	public StreamExecutionEnvironment getExecutionEnvironment() {
-		return environment;
-	}
-
-	public ExecutionConfig getExecutionConfig() {
-		return environment.getConfig();
-	}
-
-	/**
-	 * Creates a new {@link DataStream} by merging {@link DataStream} outputs of
-	 * the same type with each other. The DataStreams merged using this operator
-	 * will be transformed simultaneously.
-	 * 
-	 * @param streams
-	 *            The DataStreams to union output with.
-	 * @return The {@link DataStream}.
-	 */
-	@SafeVarargs
-	public final DataStream<T> union(DataStream<T>... streams) {
-		List<StreamTransformation<T>> unionedTransforms = new ArrayList<>();
-		unionedTransforms.add(this.transformation);
-
-		Collection<StreamTransformation<?>> thisPredecessors = this.getTransformation().getTransitivePredecessors();
-
-		for (DataStream<T> newStream : streams) {
-			if (!(newStream.getParallelism() == this.getParallelism())) {
-				throw new UnsupportedClassVersionError(
-						"DataStream can only be unioned with DataStreams of the same parallelism. " +
-								"This Stream: " + this.getTransformation() +
-								", other stream: " + newStream.getTransformation());
-			}
-			if (!getType().equals(newStream.getType())) {
-				throw new IllegalArgumentException("Cannot union streams of different types: "
-						+ getType() + " and " + newStream.getType());
-			}
-			
-			Collection<StreamTransformation<?>> predecessors = newStream.getTransformation().getTransitivePredecessors();
-
-			if (predecessors.contains(this.transformation) || thisPredecessors.contains(newStream.getTransformation())) {
-				throw new UnsupportedOperationException("A DataStream cannot be unioned with itself");
-			}
-			unionedTransforms.add(newStream.getTransformation());
-		}
-		return new DataStream<T>(this.environment, new UnionTransformation<T>(unionedTransforms));
-	}
-
-	/**
-	 * Operator used for directing tuples to specific named outputs using an
-	 * {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}.
-	 * Calling this method on an operator creates a new {@link SplitStream}.
-	 * 
-	 * @param outputSelector
-	 *            The user defined
-	 *            {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}
-	 *            for directing the tuples.
-	 * @return The {@link SplitStream}
-	 */
-	public SplitStream<T> split(OutputSelector<T> outputSelector) {
-		return new SplitStream<T>(this, clean(outputSelector));
-	}
-
-	/**
-	 * Creates a new {@link ConnectedStreams} by connecting
-	 * {@link DataStream} outputs of (possible) different types with each other.
-	 * The DataStreams connected using this operator can be used with
-	 * CoFunctions to apply joint transformations.
-	 * 
-	 * @param dataStream
-	 *            The DataStream with which this stream will be connected.
-	 * @return The {@link ConnectedStreams}.
-	 */
-	public <R> ConnectedStreams<T, R> connect(DataStream<R> dataStream) {
-		return new ConnectedStreams<T, R>(environment, this, dataStream);
-	}
-
-	/**
-	 * 
-	 * It creates a new {@link KeyedStream} that uses the provided key for partitioning
-	 * its operator states. 
-	 *
-	 * @param key
-	 *            The KeySelector to be used for extracting the key for partitioning
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
-	 */
-	public <K> KeyedStream<T, K> keyBy(KeySelector<T, K> key) {
-		return new KeyedStream<T, K>(this, clean(key));
-	}
-
-	/**
-	 * Partitions the operator state of a {@link DataStream} by the given key positions. 
-	 *
-	 * @param fields
-	 *            The position of the fields on which the {@link DataStream}
-	 *            will be grouped.
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
-	 */
-	public KeyedStream<T, Tuple> keyBy(int... fields) {
-		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return keyBy(KeySelectorUtil.getSelectorForArray(fields, getType()));
-		} else {
-			return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
-		}
-	}
-
-	/**
-	 * Partitions the operator state of a {@link DataStream}using field expressions. 
-	 * A field expression is either the name of a public field or a getter method with parentheses
-	 * of the {@link DataStream}S underlying type. A dot can be used to drill
-	 * down into objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param fields
-	 *            One or more field expressions on which the state of the {@link DataStream} operators will be
-	 *            partitioned.
-	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
-	 **/
-	public KeyedStream<T, Tuple> keyBy(String... fields) {
-		return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
-	}
-
-	private KeyedStream<T, Tuple> keyBy(Keys<T> keys) {
-		return new KeyedStream<T, Tuple>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
-				getType(), getExecutionConfig())));
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output is
-	 * partitioned hashing on the given fields. This setting only
-	 * effects the how the outputs will be distributed between the parallel
-	 * instances of the next processing operator.
-	 *
-	 * @param fields The tuple fields that should be used for partitioning
-	 * @return The partitioned DataStream
-	 *
-	 */
-	public DataStream<T> partitionByHash(int... fields) {
-		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
-			return partitionByHash(KeySelectorUtil.getSelectorForArray(fields, getType()));
-		} else {
-			return partitionByHash(new Keys.ExpressionKeys<T>(fields, getType()));
-		}
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output is
-	 * partitioned hashing on the given fields. This setting only
-	 * effects the how the outputs will be distributed between the parallel
-	 * instances of the next processing operator.
-	 *
-	 * @param fields The tuple fields that should be used for partitioning
-	 * @return The partitioned DataStream
-	 *
-	 */
-	public DataStream<T> partitionByHash(String... fields) {
-		return partitionByHash(new Keys.ExpressionKeys<T>(fields, getType()));
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output is
-	 * partitioned using the given {@link KeySelector}. This setting only
-	 * effects the how the outputs will be distributed between the parallel
-	 * instances of the next processing operator.
-	 *
-	 * @param keySelector The function that extracts the key from an element in the Stream
-	 * @return The partitioned DataStream
-	 */
-	public DataStream<T> partitionByHash(KeySelector<T, ?> keySelector) {
-		return setConnectionType(new HashPartitioner<T>(clean(keySelector)));
-	}
-
-	//private helper method for partitioning
-	private DataStream<T> partitionByHash(Keys<T> keys) {
-		KeySelector<T, ?> keySelector = clean(KeySelectorUtil.getSelectorForKeys(
-				keys,
-				getType(),
-				getExecutionConfig()));
-
-		return setConnectionType(new HashPartitioner<T>(keySelector));
-	}
-
-	/**
-	 * Partitions a tuple DataStream on the specified key fields using a custom partitioner.
-	 * This method takes the key position to partition on, and a partitioner that accepts the key type.
-	 * <p>
-	 * Note: This method works only on single field keys.
-	 *
-	 * @param partitioner The partitioner to assign partitions to keys.
-	 * @param field The field index on which the DataStream is to partitioned.
-	 * @return The partitioned DataStream.
-	 */
-	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, int field) {
-		Keys.ExpressionKeys<T> outExpressionKeys = new Keys.ExpressionKeys<T>(new int[]{field}, getType());
-		return partitionCustom(partitioner, outExpressionKeys);
-	}
-
-	/**
-	 * Partitions a POJO DataStream on the specified key fields using a custom partitioner.
-	 * This method takes the key expression to partition on, and a partitioner that accepts the key type.
-	 * <p>
-	 * Note: This method works only on single field keys.
-	 *
-	 * @param partitioner The partitioner to assign partitions to keys.
-	 * @param field The field index on which the DataStream is to partitioned.
-	 * @return The partitioned DataStream.
-	 */
-	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, String field) {
-		Keys.ExpressionKeys<T> outExpressionKeys = new Keys.ExpressionKeys<T>(new String[]{field}, getType());
-		return partitionCustom(partitioner, outExpressionKeys);
-	}
-
-
-	/**
-	 * Partitions a DataStream on the key returned by the selector, using a custom partitioner.
-	 * This method takes the key selector to get the key to partition on, and a partitioner that
-	 * accepts the key type.
-	 * <p>
-	 * Note: This method works only on single field keys, i.e. the selector cannot return tuples
-	 * of fields.
-	 *
-	 * @param partitioner
-	 * 		The partitioner to assign partitions to keys.
-	 * @param keySelector
-	 * 		The KeySelector with which the DataStream is partitioned.
-	 * @return The partitioned DataStream.
-	 * @see KeySelector
-	 */
-	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, KeySelector<T, K> keySelector) {
-		return setConnectionType(new CustomPartitionerWrapper<K, T>(clean(partitioner),
-				clean(keySelector)));
-	}
-
-	//	private helper method for custom partitioning
-	private <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, Keys<T> keys) {
-		KeySelector<T, K> keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig());
-
-		return setConnectionType(
-				new CustomPartitionerWrapper<K, T>(
-						clean(partitioner),
-						clean(keySelector)));
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output tuples
-	 * are broadcasted to every parallel instance of the next component.
-	 *
-	 * <p>
-	 * This setting only effects the how the outputs will be distributed between
-	 * the parallel instances of the next processing operator.
-	 * 
-	 * @return The DataStream with broadcast partitioning set.
-	 */
-	public DataStream<T> broadcast() {
-		return setConnectionType(new BroadcastPartitioner<T>());
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output tuples
-	 * are shuffled uniformly randomly to the next component.
-	 *
-	 * <p>
-	 * This setting only effects the how the outputs will be distributed between
-	 * the parallel instances of the next processing operator.
-	 * 
-	 * @return The DataStream with shuffle partitioning set.
-	 */
-	public DataStream<T> shuffle() {
-		return setConnectionType(new ShufflePartitioner<T>());
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output tuples
-	 * are forwarded to the local subtask of the next component (whenever
-	 * possible).
-	 *
-	 * <p>
-	 * This setting only effects the how the outputs will be distributed between
-	 * the parallel instances of the next processing operator.
-	 * 
-	 * @return The DataStream with forward partitioning set.
-	 */
-	public DataStream<T> forward() {
-		return setConnectionType(new ForwardPartitioner<T>());
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output tuples
-	 * are distributed evenly to instances of the next component in a Round-robin
-	 * fashion.
-	 *
-	 * <p>
-	 * This setting only effects the how the outputs will be distributed between
-	 * the parallel instances of the next processing operator.
-	 * 
-	 * @return The DataStream with rebalance partitioning set.
-	 */
-	public DataStream<T> rebalance() {
-		return setConnectionType(new RebalancePartitioner<T>());
-	}
-
-	/**
-	 * Sets the partitioning of the {@link DataStream} so that the output values
-	 * all go to the first instance of the next processing operator. Use this
-	 * setting with care since it might cause a serious performance bottleneck
-	 * in the application.
-	 * 
-	 * @return The DataStream with shuffle partitioning set.
-	 */
-	public DataStream<T> global() {
-		return setConnectionType(new GlobalPartitioner<T>());
-	}
-
-	/**
-	 * Initiates an iterative part of the program that feeds back data streams.
-	 * The iterative part needs to be closed by calling
-	 * {@link IterativeStream#closeWith(DataStream)}. The transformation of
-	 * this IterativeStream will be the iteration head. The data stream
-	 * given to the {@link IterativeStream#closeWith(DataStream)} method is
-	 * the data stream that will be fed back and used as the input for the
-	 * iteration head. The user can also use different feedback type than the
-	 * input of the iteration and treat the input and feedback streams as a
-	 * {@link ConnectedStreams} be calling
-	 * {@link IterativeStream#withFeedbackType(TypeInformation)}
-	 * <p>
-	 * A common usage pattern for streaming iterations is to use output
-	 * splitting to send a part of the closing data stream to the head. Refer to
-	 * {@link #split(OutputSelector)} for more information.
-	 * <p>
-	 * The iteration edge will be partitioned the same way as the first input of
-	 * the iteration head unless it is changed in the
-	 * {@link IterativeStream#closeWith(DataStream)} call.
-	 * <p>
-	 * By default a DataStream with iteration will never terminate, but the user
-	 * can use the maxWaitTime parameter to set a max waiting time for the
-	 * iteration head. If no data received in the set time, the stream
-	 * terminates.
-	 * 
-	 * @return The iterative data stream created.
-	 */
-	public IterativeStream<T> iterate() {
-		return new IterativeStream<T>(this, 0);
-	}
-
-	/**
-	 * Initiates an iterative part of the program that feeds back data streams.
-	 * The iterative part needs to be closed by calling
-	 * {@link IterativeStream#closeWith(DataStream)}. The transformation of
-	 * this IterativeStream will be the iteration head. The data stream
-	 * given to the {@link IterativeStream#closeWith(DataStream)} method is
-	 * the data stream that will be fed back and used as the input for the
-	 * iteration head. The user can also use different feedback type than the
-	 * input of the iteration and treat the input and feedback streams as a
-	 * {@link ConnectedStreams} be calling
-	 * {@link IterativeStream#withFeedbackType(TypeInformation)}
-	 * <p>
-	 * A common usage pattern for streaming iterations is to use output
-	 * splitting to send a part of the closing data stream to the head. Refer to
-	 * {@link #split(OutputSelector)} for more information.
-	 * <p>
-	 * The iteration edge will be partitioned the same way as the first input of
-	 * the iteration head unless it is changed in the
-	 * {@link IterativeStream#closeWith(DataStream)} call.
-	 * <p>
-	 * By default a DataStream with iteration will never terminate, but the user
-	 * can use the maxWaitTime parameter to set a max waiting time for the
-	 * iteration head. If no data received in the set time, the stream
-	 * terminates.
-	 * 
-	 * @param maxWaitTimeMillis
-	 *            Number of milliseconds to wait between inputs before shutting
-	 *            down
-	 * 
-	 * @return The iterative data stream created.
-	 */
-	public IterativeStream<T> iterate(long maxWaitTimeMillis) {
-		return new IterativeStream<T>(this, maxWaitTimeMillis);
-	}
-
-	/**
-	 * Applies a Map transformation on a {@link DataStream}. The transformation
-	 * calls a {@link MapFunction} for each element of the DataStream. Each
-	 * MapFunction call returns exactly one element. The user can also extend
-	 * {@link RichMapFunction} to gain access to other features provided by the
-	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * 
-	 * @param mapper
-	 *            The MapFunction that is called for each element of the
-	 *            DataStream.
-	 * @param <R>
-	 *            output type
-	 * @return The transformed {@link DataStream}.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> map(MapFunction<T, R> mapper) {
-
-		TypeInformation<R> outType = TypeExtractor.getMapReturnTypes(clean(mapper), getType(),
-				Utils.getCallLocationName(), true);
-
-		return transform("Map", outType, new StreamMap<T, R>(clean(mapper)));
-	}
-
-	/**
-	 * Applies a FlatMap transformation on a {@link DataStream}. The
-	 * transformation calls a {@link FlatMapFunction} for each element of the
-	 * DataStream. Each FlatMapFunction call can return any number of elements
-	 * including none. The user can also extend {@link RichFlatMapFunction} to
-	 * gain access to other features provided by the
-	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * 
-	 * @param flatMapper
-	 *            The FlatMapFunction that is called for each element of the
-	 *            DataStream
-	 * 
-	 * @param <R>
-	 *            output type
-	 * @return The transformed {@link DataStream}.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> flatMap(FlatMapFunction<T, R> flatMapper) {
-
-		TypeInformation<R> outType = TypeExtractor.getFlatMapReturnTypes(clean(flatMapper),
-				getType(), Utils.getCallLocationName(), true);
-
-		return transform("Flat Map", outType, new StreamFlatMap<T, R>(clean(flatMapper)));
-
-	}
-
-	/**
-	 * Applies a Filter transformation on a {@link DataStream}. The
-	 * transformation calls a {@link FilterFunction} for each element of the
-	 * DataStream and retains only those element for which the function returns
-	 * true. Elements for which the function returns false are filtered. The
-	 * user can also extend {@link RichFilterFunction} to gain access to other
-	 * features provided by the
-	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
-	 * 
-	 * @param filter
-	 *            The FilterFunction that is called for each element of the
-	 *            DataStream.
-	 * @return The filtered DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> filter(FilterFunction<T> filter) {
-		return transform("Filter", getType(), new StreamFilter<T>(clean(filter)));
-
-	}
-
-	/**
-	 * Initiates a Project transformation on a {@link Tuple} {@link DataStream}.<br/>
-	 * <b>Note: Only Tuple DataStreams can be projected.</b>
-	 *
-	 * <p>
-	 * The transformation projects each Tuple of the DataSet onto a (sub)set of
-	 * fields.
-	 * 
-	 * @param fieldIndexes
-	 *            The field indexes of the input tuples that are retained. The
-	 *            order of fields in the output tuple corresponds to the order
-	 *            of field indexes.
-	 * @return The projected DataStream
-	 * 
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <R extends Tuple> SingleOutputStreamOperator<R, ?> project(int... fieldIndexes) {
-		return new StreamProjection<T>(this, fieldIndexes).projectTupleX();
-	}
-
-	/**
-	 * Creates a join operation. See {@link CoGroupedStreams} for an example of how the keys
-	 * and window can be specified.
-	 */
-	public <T2> CoGroupedStreams<T, T2> coGroup(DataStream<T2> otherStream) {
-		return new CoGroupedStreams<>(this, otherStream);
-	}
-
-	/**
-	 * Creates a join operation. See {@link JoinedStreams} for an example of how the keys
-	 * and window can be specified.
-	 */
-	public <T2> JoinedStreams<T, T2> join(DataStream<T2> otherStream) {
-		return new JoinedStreams<>(this, otherStream);
-	}
-
-	/**
-	 * Windows this {@code DataStream} 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
-	 *
-	 * <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).
-	 *
-	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
-	 *
-	 * @param size The size of the window.
-	 */
-	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size) {
-		return windowAll(TumblingTimeWindows.of(size));
-	}
-
-	/**
-	 * Windows this {@code DataStream} 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)}
-	 *
-	 * <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 size The size of the window.
-	 */
-	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size, AbstractTime slide) {
-		return windowAll(SlidingTimeWindows.of(size, slide));
-	}
-
-	/**
-	 * Windows this {@code DataStream} into tumbling count windows.
-	 *
-	 * <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 size The size of the windows in number of elements.
-	 */
-	public AllWindowedStream<T, GlobalWindow> countWindowAll(long size) {
-		return windowAll(GlobalWindows.create()).trigger(PurgingTrigger.of(CountTrigger.of(size)));
-	}
-
-	/**
-	 * Windows this {@code DataStream} into sliding count windows.
-	 *
-	 * <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 size The size of the windows in number of elements.
-	 * @param slide The slide interval in number of elements.
-	 */
-	public AllWindowedStream<T, GlobalWindow> countWindowAll(long size, long slide) {
-		return windowAll(GlobalWindows.create())
-				.evictor(CountEvictor.of(size))
-				.trigger(CountTrigger.of(slide));
-	}
-
-	/**
-	 * 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> AllWindowedStream<T, W> windowAll(WindowAssigner<? super T, W> assigner) {
-		return new AllWindowedStream<>(this, assigner);
-	}
-
-	/**
-	 * Extracts a timestamp from an element and assigns it as the internal timestamp of that element.
-	 * The internal timestamps are, for example, used to to event-time window operations.
-	 *
-	 * <p>
-	 * If you know that the timestamps are strictly increasing you can use an
-	 * {@link org.apache.flink.streaming.api.functions.AscendingTimestampExtractor}. Otherwise,
-	 * you should provide a {@link TimestampExtractor} that also implements
-	 * {@link TimestampExtractor#getCurrentWatermark()} to keep track of watermarks.
-	 *
-	 * @see org.apache.flink.streaming.api.watermark.Watermark
-	 *
-	 * @param extractor The TimestampExtractor that is called for each element of the DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> assignTimestamps(TimestampExtractor<T> extractor) {
-		// match parallelism to input, otherwise dop=1 sources could lead to some strange
-		// behaviour: the watermark will creep along very slowly because the elements
-		// from the source go to each extraction operator round robin.
-		int inputParallelism = getTransformation().getParallelism();
-		ExtractTimestampsOperator<T> operator = new ExtractTimestampsOperator<>(clean(extractor));
-		return transform("ExtractTimestamps", getTransformation().getOutputType(), operator)
-				.setParallelism(inputParallelism);
-	}
-
-	/**
-	 * Writes a DataStream to the standard output stream (stdout).
-	 *
-	 * <p>
-	 * For each element of the DataStream the result of
-	 * {@link Object#toString()} is written.
-	 * 
-	 * @return The closed DataStream.
-	 */
-	public DataStreamSink<T> print() {
-		PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>();
-		return addSink(printFunction);
-	}
-
-	/**
-	 * Writes a DataStream to the standard output stream (stderr).
-	 *
-	 * <p>
-	 * For each element of the DataStream the result of
-	 * {@link Object#toString()} is written.
-	 * 
-	 * @return The closed DataStream.
-	 */
-	public DataStreamSink<T> printToErr() {
-		PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>(true);
-		return addSink(printFunction);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in text format.
-	 *
-	 * <p>
-	 * For every element of the DataStream the result of {@link Object#toString()}
-	 * is written.
-	 * 
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * 
-	 * @return the closed DataStream.
-	 */
-	public DataStreamSink<T> writeAsText(String path) {
-		return write(new TextOutputFormat<T>(new Path(path)), 0L);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in text format. The
-	 * writing is performed periodically, in every millis milliseconds.
-	 *
-	 * <p>
-	 * For every element of the DataStream the result of {@link Object#toString()}
-	 * is written.
-	 * 
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * @param millis
-	 *            the file update frequency
-	 * 
-	 * @return the closed DataStream
-	 */
-	public DataStreamSink<T> writeAsText(String path, long millis) {
-		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
-		return write(tof, millis);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in text format.
-	 *
-	 * <p>
-	 * For every element of the DataStream the result of {@link Object#toString()}
-	 * is written.
-	 * 
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * @param writeMode
-	 *            Control the behavior for existing files. Options are
-	 *            NO_OVERWRITE and OVERWRITE.
-	 * 
-	 * @return the closed DataStream.
-	 */
-	public DataStreamSink<T> writeAsText(String path, WriteMode writeMode) {
-		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
-		tof.setWriteMode(writeMode);
-		return write(tof, 0L);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in text format.
-	 *
-	 * <p>
-	 * For every element of the DataStream the result of {@link Object#toString()}
-	 * is written.
-	 * 
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * @param writeMode
-	 *            Controls the behavior for existing files. Options are
-	 *            NO_OVERWRITE and OVERWRITE.
-	 * @param millis
-	 *            the file update frequency
-	 * 
-	 * @return the closed DataStream.
-	 */
-	public DataStreamSink<T> writeAsText(String path, WriteMode writeMode, long millis) {
-		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
-		tof.setWriteMode(writeMode);
-		return write(tof, millis);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in csv format.
-	 *
-	 * <p>
-	 * For every field of an element of the DataStream the result of {@link Object#toString()}
-	 * is written. This method can only be used on data streams of tuples.
-	 * 
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * 
-	 * @return the closed DataStream
-	 */
-	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path) {
-		Preconditions.checkArgument(getType().isTupleType(),
-				"The writeAsCsv() method can only be used on data sets of tuples.");
-		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
-				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
-		return write((OutputFormat<T>) of, 0L);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in csv format. The
-	 * writing is performed periodically, in every millis milliseconds.
-	 *
-	 * <p>
-	 * For every field of an element of the DataStream the result of {@link Object#toString()}
-	 * is written. This method can only be used on data streams of tuples.
-	 *
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * @param millis
-	 *            the file update frequency
-	 * 
-	 * @return the closed DataStream
-	 */
-	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, long millis) {
-		Preconditions.checkArgument(getType().isTupleType(),
-				"The writeAsCsv() method can only be used on data sets of tuples.");
-		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
-				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
-		return write((OutputFormat<T>) of, millis);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in csv format.
-	 *
-	 * <p>
-	 * For every field of an element of the DataStream the result of {@link Object#toString()}
-	 * is written. This method can only be used on data streams of tuples.
-	 * 
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * @param writeMode
-	 *            Controls the behavior for existing files. Options are
-	 *            NO_OVERWRITE and OVERWRITE.
-	 * 
-	 * @return the closed DataStream
-	 */
-	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, WriteMode writeMode) {
-		Preconditions.checkArgument(getType().isTupleType(),
-				"The writeAsCsv() method can only be used on data sets of tuples.");
-		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
-				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
-		if (writeMode != null) {
-			of.setWriteMode(writeMode);
-		}
-		return write((OutputFormat<T>) of, 0L);
-	}
-
-	/**
-	 * Writes a DataStream to the file specified by path in csv format. The
-	 * writing is performed periodically, in every millis milliseconds.
-	 *
-	 * <p>
-	 * For every field of an element of the DataStream the result of {@link Object#toString()}
-	 * is written. This method can only be used on data streams of tuples.
-	 * 
-	 * @param path
-	 *            the path pointing to the location the text file is written to
-	 * @param writeMode
-	 *            Controls the behavior for existing files. Options are
-	 *            NO_OVERWRITE and OVERWRITE.
-	 * @param millis
-	 *            the file update frequency
-	 * 
-	 * @return the closed DataStream
-	 */
-	@SuppressWarnings("unchecked")
-	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, WriteMode writeMode,
-			long millis) {
-		Preconditions.checkArgument(getType().isTupleType(),
-				"The writeAsCsv() method can only be used on data sets of tuples.");
-		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
-				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
-		if (writeMode != null) {
-			of.setWriteMode(writeMode);
-		}
-		return write((OutputFormat<T>) of, millis);
-	}
-
-	/**
-	 * Writes the DataStream to a socket as a byte array. The format of the
-	 * output is specified by a {@link SerializationSchema}.
-	 * 
-	 * @param hostName
-	 *            host of the socket
-	 * @param port
-	 *            port of the socket
-	 * @param schema
-	 *            schema for serialization
-	 * @return the closed DataStream
-	 */
-	public DataStreamSink<T> writeToSocket(String hostName, int port, SerializationSchema<T, byte[]> schema) {
-		DataStreamSink<T> returnStream = addSink(new SocketClientSink<T>(hostName, port, schema, 0));
-		returnStream.setParallelism(1); // It would not work if multiple instances would connect to the same port
-		return returnStream;
-	}
-	
-	/**
-	 * Writes the dataStream into an output, described by an OutputFormat.
-	 * 
-	 * @param format The output format
-	 * @param millis the write frequency
-	 * @return The closed DataStream
-	 */
-	public DataStreamSink<T> write(OutputFormat<T> format, long millis) {
-		return addSink(new FileSinkFunctionByMillis<T>(format, millis));
-	}
-
-	/**
-	 * Method for passing user defined operators along with the type
-	 * information that will transform the DataStream.
-	 * 
-	 * @param operatorName
-	 *            name of the operator, for logging purposes
-	 * @param outTypeInfo
-	 *            the output type of the operator
-	 * @param operator
-	 *            the object containing the transformation logic
-	 * @param <R>
-	 *            type of the return stream
-	 * @return the data stream constructed
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName, TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
-
-		// read the output type of the input Transform to coax out errors about MissingTypeInfo
-		transformation.getOutputType();
-
-		OneInputTransformation<T, R> resultTransform = new OneInputTransformation<T, R>(
-				this.transformation,
-				operatorName,
-				operator,
-				outTypeInfo,
-				environment.getParallelism());
-
-		@SuppressWarnings({ "unchecked", "rawtypes" })
-		SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment, resultTransform);
-
-		getExecutionEnvironment().addOperator(resultTransform);
-
-		return returnStream;
-	}
-
-	/**
-	 * Internal function for setting the partitioner for the DataStream
-	 *
-	 * @param partitioner
-	 *            Partitioner to set.
-	 * @return The modified DataStream.
-	 */
-	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
-		return new DataStream<T>(this.getExecutionEnvironment(), new PartitionTransformation<T>(this.getTransformation(), partitioner));
-	}
-
-	/**
-	 * Adds the given sink to this DataStream. Only streams with sinks added
-	 * will be executed once the {@link StreamExecutionEnvironment#execute()}
-	 * method is called.
-	 * 
-	 * @param sinkFunction
-	 *            The object containing the sink's invoke function.
-	 * @return The closed DataStream.
-	 */
-	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
-
-		// read the output type of the input Transform to coax out errors about MissingTypeInfo
-		transformation.getOutputType();
-
-		// configure the type if needed
-		if (sinkFunction instanceof InputTypeConfigurable) {
-			((InputTypeConfigurable) sinkFunction).setInputType(getType(), getExecutionConfig() );
-		}
-
-		StreamSink<T> sinkOperator = new StreamSink<T>(clean(sinkFunction));
-
-		DataStreamSink<T> sink = new DataStreamSink<T>(this, sinkOperator);
-
-		getExecutionEnvironment().addOperator(sink.getTransformation());
-		return sink;
-	}
-
-	/**
-	 * Returns the {@link StreamTransformation} that represents the operation that logically creates
-	 * this {@link DataStream}.
-	 *
-	 * @return The Transformation
-	 */
-	public StreamTransformation<T> getTransformation() {
-		return transformation;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
deleted file mode 100644
index 24104ad..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.StreamSink;
-import org.apache.flink.streaming.api.transformations.SinkTransformation;
-
-/**
- * A Stream Sink. This is used for emitting elements from a streaming topology.
- *
- * @param <T> The type of the elements in the Stream
- */
-public class DataStreamSink<T> {
-
-	SinkTransformation<T> transformation;
-
-	@SuppressWarnings("unchecked")
-	protected DataStreamSink(DataStream<T> inputStream, StreamSink<T> operator) {
-		this.transformation = new SinkTransformation<T>(inputStream.getTransformation(), "Unnamed", operator, inputStream.getExecutionEnvironment().getParallelism());
-	}
-
-	/**
-	 * Returns the transformation that contains the actual sink operator of this sink.
-	 */
-	public SinkTransformation<T> getTransformation() {
-		return transformation;
-	}
-
-	/**
-	 * Sets the name of this sink. This name is
-	 * used by the visualization and logging during runtime.
-	 *
-	 * @return The named sink.
-	 */
-	public DataStreamSink<T> name(String name) {
-		transformation.setName(name);
-		return this;
-	}
-
-	/**
-	 * Sets the parallelism for this sink. The degree must be higher than zero.
-	 *
-	 * @param parallelism The parallelism for this sink.
-	 * @return The sink with set parallelism.
-	 */
-	public DataStreamSink<T> setParallelism(int parallelism) {
-		transformation.setParallelism(parallelism);
-		return this;
-	}
-
-	/**
-	 * Turns off chaining for this operator so thread co-location will not be
-	 * used as an optimization.
-	 *
-	 * <p>
-	 * Chaining can be turned off for the whole
-	 * job by {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#disableOperatorChaining()}
-	 * however it is not advised for performance considerations.
-	 *
-	 * @return The sink with chaining disabled
-	 */
-	public DataStreamSink<T> disableChaining() {
-		this.transformation.setChainingStrategy(ChainingStrategy.NEVER);
-		return this;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
deleted file mode 100644
index d2e04a7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.transformations.SourceTransformation;
-
-/**
- * The DataStreamSource represents the starting point of a DataStream.
- * 
- * @param <T> Type of the elements in the DataStream created from the this source.
- */
-public class DataStreamSource<T> extends SingleOutputStreamOperator<T, DataStreamSource<T>> {
-
-	boolean isParallel;
-
-	public DataStreamSource(StreamExecutionEnvironment environment,
-			TypeInformation<T> outTypeInfo, StreamSource<T> operator,
-			boolean isParallel, String sourceName) {
-		super(environment, new SourceTransformation<T>(sourceName, operator, outTypeInfo, environment.getParallelism()));
-
-		this.isParallel = isParallel;
-		if (!isParallel) {
-			setParallelism(1);
-		}
-	}
-
-	@Override
-	public DataStreamSource<T> setParallelism(int parallelism) {
-		if (parallelism > 1 && !isParallel) {
-			throw new IllegalArgumentException("Source: " + transformation.getId() + " is not a parallel source");
-		} else {
-			return (DataStreamSource<T>) super.setParallelism(parallelism);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
deleted file mode 100644
index 346bef9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
-import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-
-import java.util.Collection;
-
-/**
- * The iterative data stream represents the start of an iteration in a {@link DataStream}.
- * 
- * @param <T> Type of the elements in this Stream
- */
-public class IterativeStream<T> extends SingleOutputStreamOperator<T, IterativeStream<T>> {
-
-	// We store these so that we can create a co-iteration if we need to
-	private DataStream<T> originalInput;
-	private long maxWaitTime;
-	
-	protected IterativeStream(DataStream<T> dataStream, long maxWaitTime) {
-		super(dataStream.getExecutionEnvironment(),
-				new FeedbackTransformation<T>(dataStream.getTransformation(), maxWaitTime));
-		this.originalInput = dataStream;
-		this.maxWaitTime = maxWaitTime;
-		setBufferTimeout(dataStream.environment.getBufferTimeout());
-	}
-
-	/**
-	 * Closes the iteration. This method defines the end of the iterative
-	 * program part that will be fed back to the start of the iteration.
-	 *
-	 * <p>
-	 * A common usage pattern for streaming iterations is to use output
-	 * splitting to send a part of the closing data stream to the head. Refer to
-	 * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)}
-	 * for more information.
-	 * 
-	 * @param feedbackStream
-	 *            {@link DataStream} that will be used as input to the iteration
-	 *            head.
-	 *
-	 * @return The feedback stream.
-	 * 
-	 */
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	public DataStream<T> closeWith(DataStream<T> feedbackStream) {
-
-		Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
-
-		if (!predecessors.contains(this.transformation)) {
-			throw new UnsupportedOperationException(
-					"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
-		}
-
-		((FeedbackTransformation) getTransformation()).addFeedbackEdge(feedbackStream.getTransformation());
-
-		return feedbackStream;
-	}
-
-	/**
-	 * Changes the feedback type of the iteration and allows the user to apply
-	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * For type safety the user needs to define the feedback type
-	 * 
-	 * @param feedbackTypeString
-	 *            String describing the type information of the feedback stream.
-	 * @return A {@link ConnectedIterativeStreams}.
-	 */
-	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(String feedbackTypeString) {
-		return withFeedbackType(TypeInfoParser.<F> parse(feedbackTypeString));
-	}
-
-	/**
-	 * Changes the feedback type of the iteration and allows the user to apply
-	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * For type safety the user needs to define the feedback type
-	 * 
-	 * @param feedbackTypeClass
-	 *            Class of the elements in the feedback stream.
-	 * @return A {@link ConnectedIterativeStreams}.
-	 */
-	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(Class<F> feedbackTypeClass) {
-		return withFeedbackType(TypeExtractor.getForClass(feedbackTypeClass));
-	}
-
-	/**
-	 * Changes the feedback type of the iteration and allows the user to apply
-	 * co-transformations on the input and feedback stream, as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * For type safety the user needs to define the feedback type
-	 * 
-	 * @param feedbackType
-	 *            The type information of the feedback stream.
-	 * @return A {@link ConnectedIterativeStreams}.
-	 */
-	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(TypeInformation<F> feedbackType) {
-		return new ConnectedIterativeStreams<T, F>(originalInput, feedbackType, maxWaitTime);
-	}
-	
-	/**
-	 * The {@link ConnectedIterativeStreams} represent a start of an
-	 * iterative part of a streaming program, where the original input of the
-	 * iteration and the feedback of the iteration are connected as in a
-	 * {@link ConnectedStreams}.
-	 *
-	 * <p>
-	 * The user can distinguish between the two inputs using co-transformation,
-	 * thus eliminating the need for mapping the inputs and outputs to a common
-	 * type.
-	 * 
-	 * @param <I>
-	 *            Type of the input of the iteration
-	 * @param <F>
-	 *            Type of the feedback of the iteration
-	 */
-	public static class ConnectedIterativeStreams<I, F> extends ConnectedStreams<I, F> {
-
-		private CoFeedbackTransformation<F> coFeedbackTransformation;
-
-		public ConnectedIterativeStreams(DataStream<I> input,
-				TypeInformation<F> feedbackType,
-				long waitTime) {
-			super(input.getExecutionEnvironment(),
-					input,
-					new DataStream<F>(input.getExecutionEnvironment(),
-							new CoFeedbackTransformation<F>(input.getParallelism(),
-									feedbackType,
-									waitTime)));
-			this.coFeedbackTransformation = (CoFeedbackTransformation<F>) getSecondInput().getTransformation();
-		}
-
-		/**
-		 * Closes the iteration. This method defines the end of the iterative
-		 * program part that will be fed back to the start of the iteration as
-		 * the second input in the {@link ConnectedStreams}.
-		 * 
-		 * @param feedbackStream
-		 *            {@link DataStream} that will be used as second input to
-		 *            the iteration head.
-		 * @return The feedback stream.
-		 * 
-		 */
-		@SuppressWarnings({ "rawtypes", "unchecked" })
-		public DataStream<F> closeWith(DataStream<F> feedbackStream) {
-
-			Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
-
-			if (!predecessors.contains(this.coFeedbackTransformation)) {
-				throw new UnsupportedOperationException(
-						"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
-			}
-
-			coFeedbackTransformation.addFeedbackEdge(feedbackStream.getTransformation());
-
-			return feedbackStream;
-		}
-		
-		private UnsupportedOperationException groupingException = new UnsupportedOperationException(
-				"Cannot change the input partitioning of an iteration head directly. Apply the partitioning on the input and feedback streams instead.");
-		
-		@Override
-		public ConnectedStreams<I, F> keyBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> keyBy(String field1, String field2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> keyBy(String[] fields1, String[] fields2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> keyBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
-
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(int keyPosition1, int keyPosition2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(String field1, String field2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(String[] fields1, String[] fields2) {throw groupingException;}
-		
-		@Override
-		public ConnectedStreams<I, F> partitionByHash(KeySelector<I, ?> keySelector1, KeySelector<F, ?> keySelector2) {throw groupingException;}
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
deleted file mode 100644
index cff9355..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.translation.WrappingFunction;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion;
-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.util.Collector;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- *{@code JoinedStreams} represents two {@link DataStream DataStreams} that have been joined.
- * A streaming join operation is evaluated over elements in a window.
- *
- * <p>
- * To finalize the join operation you also need to specify a {@link KeySelector} for
- * both the first and second input and a {@link WindowAssigner}.
- *
- * <p>
- * Note: Right now, the the join is being evaluated in memory so you need to ensure that the number
- * of elements per key does not get too high. Otherwise the JVM might crash.
- *
- * <p>
- * Example:
- *
- * <pre> {@code
- * DataStream<Tuple2<String, Integer>> one = ...;
- * DataStream<Tuple2<String, Integer>> twp = ...;
- *
- * DataStream<T> result = one.join(two)
- *     .where(new MyFirstKeySelector())
- *     .equalTo(new MyFirstKeySelector())
- *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
- *     .apply(new MyJoinFunction());
- * } </pre>
- */
-public class JoinedStreams<T1, T2> {
-
-	/** The first input stream */
-	private final DataStream<T1> input1;
-
-	/** The second input stream */
-	private final DataStream<T2> input2;
-
-	/**
-	 * Creates new JoinedStreams data streams, which are the first step towards building a streaming co-group.
-	 *
-	 * @param input1 The first data stream.
-	 * @param input2 The second data stream.
-	 */
-	public JoinedStreams(DataStream<T1> input1, DataStream<T2> input2) {
-		this.input1 = requireNonNull(input1);
-		this.input2 = requireNonNull(input2);
-	}
-
-	/**
-	 * Specifies a {@link KeySelector} for elements from the first input.
-	 */
-	public <KEY> Where<KEY> where(KeySelector<T1, KEY> keySelector)  {
-		TypeInformation<KEY> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
-		return new Where<>(input1.clean(keySelector), keyType);
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * CoGrouped streams that have the key for one side defined.
-	 *
-	 * @param <KEY> The type of the key.
-	 */
-	public class Where<KEY> {
-
-		private final KeySelector<T1, KEY> keySelector1;
-		private final TypeInformation<KEY> keyType;
-
-		Where(KeySelector<T1, KEY> keySelector1, TypeInformation<KEY> keyType) {
-			this.keySelector1 = keySelector1;
-			this.keyType = keyType;
-		}
-
-		/**
-		 * Specifies a {@link KeySelector} for elements from the second input.
-		 */
-		public EqualTo equalTo(KeySelector<T2, KEY> keySelector)  {
-			TypeInformation<KEY> otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
-			if (!otherKey.equals(this.keyType)) {
-				throw new IllegalArgumentException("The keys for the two inputs are not equal: " +
-						"first key = " + this.keyType + " , second key = " + otherKey);
-			}
-
-			return new EqualTo(input2.clean(keySelector));
-		}
-
-		// --------------------------------------------------------------------
-
-		/**
-		 * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs.
-		 */
-		public class EqualTo {
-
-			private final KeySelector<T2, KEY> keySelector2;
-
-			EqualTo(KeySelector<T2, KEY> keySelector2) {
-				this.keySelector2 = requireNonNull(keySelector2);
-			}
-
-			/**
-			 * Specifies the window on which the co-group operation works.
-			 */
-			public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
-				return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null);
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A join operation that has {@link KeySelector KeySelectors} defined for both inputs as
-	 * well as a {@link WindowAssigner}.
-	 *
-	 * @param <T1> Type of the elements from the first input
-	 * @param <T2> Type of the elements from the second input
-	 * @param <KEY> Type of the key. This must be the same for both inputs
-	 * @param <W> Type of {@link Window} on which the join operation works.
-	 */
-	public static class WithWindow<T1, T2, KEY, W extends Window> {
-		
-		private final DataStream<T1> input1;
-		private final DataStream<T2> input2;
-
-		private final KeySelector<T1, KEY> keySelector1;
-		private final KeySelector<T2, KEY> keySelector2;
-		private final TypeInformation<KEY> keyType;
-
-		private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
-
-		private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
-
-		private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
-
-		protected WithWindow(DataStream<T1> input1,
-				DataStream<T2> input2,
-				KeySelector<T1, KEY> keySelector1,
-				KeySelector<T2, KEY> keySelector2,
-				TypeInformation<KEY> keyType,
-				WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
-				Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
-				Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
-			
-			this.input1 = requireNonNull(input1);
-			this.input2 = requireNonNull(input2);
-
-			this.keySelector1 = requireNonNull(keySelector1);
-			this.keySelector2 = requireNonNull(keySelector2);
-			this.keyType = requireNonNull(keyType);
-			
-			this.windowAssigner = requireNonNull(windowAssigner);
-			
-			this.trigger = trigger;
-			this.evictor = evictor;
-		}
-
-		/**
-		 * Sets the {@code Trigger} that should be used to trigger window emission.
-		 */
-		public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
-			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
-					windowAssigner, newTrigger, evictor);
-		}
-
-		/**
-		 * 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 WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
-			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
-					windowAssigner, trigger, newEvictor);
-		}
-
-		/**
-		 * Completes the join operation with the user function that is executed
-		 * for each combination of elements with the same key in a window.
-		 */
-		public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function) {
-			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
-					function,
-					JoinFunction.class,
-					true,
-					true,
-					input1.getType(),
-					input2.getType(),
-					"Join",
-					false);
-
-			return apply(function, resultType);
-		}
-
-		/**
-		 * Completes the join operation with the user function that is executed
-		 * for each combination of elements with the same key in a window.
-		 */
-		public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
-			//clean the closure
-			function = input1.getExecutionEnvironment().clean(function);
-
-			return input1.coGroup(input2)
-					.where(keySelector1)
-					.equalTo(keySelector2)
-					.window(windowAssigner)
-					.trigger(trigger)
-					.evictor(evictor)
-					.apply(new FlatJoinCoGroupFunction<>(function), resultType);
-
-		}
-
-		/**
-		 * Completes the join operation with the user function that is executed
-		 * for each combination of elements with the same key in a window.
-		 */
-		public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function) {
-			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
-					function,
-					JoinFunction.class,
-					true,
-					true,
-					input1.getType(),
-					input2.getType(),
-					"Join",
-					false);
-
-			return apply(function, resultType);
-		}
-
-		/**
-		 * Completes the join operation with the user function that is executed
-		 * for each combination of elements with the same key in a window.
-		 */
-		public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
-			//clean the closure
-			function = input1.getExecutionEnvironment().clean(function);
-
-			return input1.coGroup(input2)
-					.where(keySelector1)
-					.equalTo(keySelector2)
-					.window(windowAssigner)
-					.trigger(trigger)
-					.evictor(evictor)
-					.apply(new JoinCoGroupFunction<>(function), resultType);
-
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Implementation of the functions
-	// ------------------------------------------------------------------------
-
-	/**
-	 * CoGroup function that does a nested-loop join to get the join result.
-	 */
-	private static class JoinCoGroupFunction<T1, T2, T>
-			extends WrappingFunction<JoinFunction<T1, T2, T>>
-			implements CoGroupFunction<T1, T2, T> {
-		private static final long serialVersionUID = 1L;
-
-		public JoinCoGroupFunction(JoinFunction<T1, T2, T> wrappedFunction) {
-			super(wrappedFunction);
-		}
-
-		@Override
-		public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
-			for (T1 val1: first) {
-				for (T2 val2: second) {
-					out.collect(wrappedFunction.join(val1, val2));
-				}
-			}
-		}
-	}
-
-	/**
-	 * CoGroup function that does a nested-loop join to get the join result. (FlatJoin version)
-	 */
-	private static class FlatJoinCoGroupFunction<T1, T2, T>
-			extends WrappingFunction<FlatJoinFunction<T1, T2, T>>
-			implements CoGroupFunction<T1, T2, T> {
-		private static final long serialVersionUID = 1L;
-
-		public FlatJoinCoGroupFunction(FlatJoinFunction<T1, T2, T> wrappedFunction) {
-			super(wrappedFunction);
-		}
-
-		@Override
-		public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
-			for (T1 val1: first) {
-				for (T2 val2: second) {
-					wrappedFunction.join(val1, val2, out);
-				}
-			}
-		}
-	}
-
-}


[25/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
deleted file mode 100644
index 4dbf7b8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.co;
-
-import static org.junit.Assert.fail;
-
-import java.io.Serializable;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
-import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for {@link CoStreamFlatMap}. These test that:
- *
- * <ul>
- *     <li>RichFunction methods are called correctly</li>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-public class CoStreamFlatMapTest implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	private final static class MyCoFlatMap implements CoFlatMapFunction<String, Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap1(String value, Collector<String> coll) {
-			for (int i = 0; i < value.length(); i++) {
-				coll.collect(value.substring(i, i + 1));
-			}
-		}
-
-		@Override
-		public void flatMap2(Integer value, Collector<String> coll) {
-			coll.collect(value.toString());
-		}
-	}
-
-	@Test
-	public void testCoFlatMap() throws Exception {
-		CoStreamFlatMap<String, Integer, String> operator = new CoStreamFlatMap<String, Integer, String>(new MyCoFlatMap());
-
-		TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness = new TwoInputStreamOperatorTestHarness<String, Integer, String>(operator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.open();
-
-		testHarness.processElement1(new StreamRecord<String>("abc", initialTime + 1));
-		testHarness.processElement1(new StreamRecord<String>("def", initialTime + 2));
-		testHarness.processWatermark1(new Watermark(initialTime + 2));
-		testHarness.processElement1(new StreamRecord<String>("ghi", initialTime + 3));
-
-		testHarness.processElement2(new StreamRecord<Integer>(1, initialTime + 1));
-		testHarness.processElement2(new StreamRecord<Integer>(2, initialTime + 2));
-		testHarness.processWatermark2(new Watermark(initialTime + 3));
-		testHarness.processElement2(new StreamRecord<Integer>(3, initialTime + 3));
-		testHarness.processElement2(new StreamRecord<Integer>(4, initialTime + 4));
-		testHarness.processElement2(new StreamRecord<Integer>(5, initialTime + 5));
-
-		expectedOutput.add(new StreamRecord<String>("a", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("b", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("c", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("d", initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("e", initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("f", initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("g", initialTime + 3));
-		expectedOutput.add(new StreamRecord<String>("h", initialTime + 3));
-		expectedOutput.add(new StreamRecord<String>("i", initialTime + 3));
-
-		expectedOutput.add(new StreamRecord<String>("1", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("2", initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("3", initialTime + 3));
-		expectedOutput.add(new StreamRecord<String>("4", initialTime + 4));
-		expectedOutput.add(new StreamRecord<String>("5", initialTime + 5));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	@Test
-	public void testOpenClose() throws Exception {
-		CoStreamFlatMap<String, Integer, String> operator = new CoStreamFlatMap<String, Integer, String>(new TestOpenCloseCoFlatMapFunction());
-
-		TwoInputStreamOperatorTestHarness<String, Integer, String> testHarness = new TwoInputStreamOperatorTestHarness<String, Integer, String>(operator);
-
-		long initialTime = 0L;
-
-		testHarness.open();
-
-		testHarness.processElement1(new StreamRecord<String>("Hello", initialTime));
-		testHarness.processElement2(new StreamRecord<Integer>(42, initialTime));
-
-		testHarness.close();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseCoFlatMapFunction.closeCalled);
-		Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseCoFlatMapFunction extends RichCoFlatMapFunction<String, Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public void flatMap1(String value, Collector<String> out) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			out.collect(value);
-		}
-
-		@Override
-		public void flatMap2(Integer value, Collector<String> out) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			out.collect(value.toString());
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java
deleted file mode 100644
index 28ae664..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.co;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.co.RichCoMapFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.Serializable;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-/**
- * Tests for {@link org.apache.flink.streaming.api.operators.co.CoStreamMap}. These test that:
- *
- * <ul>
- *     <li>RichFunction methods are called correctly</li>
- *     <li>Timestamps of processed elements match the input timestamp</li>
- *     <li>Watermarks are correctly forwarded</li>
- * </ul>
- */
-public class CoStreamMapTest implements Serializable {
-	private static final long serialVersionUID = 1L;
-
-	private final static class MyCoMap implements CoMapFunction<Double, Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map1(Double value) {
-			return value.toString();
-		}
-
-		@Override
-		public String map2(Integer value) {
-			return value.toString();
-		}
-	}
-
-
-	@Test
-	public void testCoMap() throws Exception {
-		CoStreamMap<Double, Integer, String> operator = new CoStreamMap<Double, Integer, String>(new MyCoMap());
-
-		TwoInputStreamOperatorTestHarness<Double, Integer, String> testHarness = new TwoInputStreamOperatorTestHarness<Double, Integer, String>(operator);
-
-		long initialTime = 0L;
-		ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<Object>();
-
-		testHarness.open();
-
-		testHarness.processElement1(new StreamRecord<Double>(1.1d, initialTime + 1));
-		testHarness.processElement1(new StreamRecord<Double>(1.2d, initialTime + 2));
-		testHarness.processElement1(new StreamRecord<Double>(1.3d, initialTime + 3));
-		testHarness.processWatermark1(new Watermark(initialTime + 3));
-		testHarness.processElement1(new StreamRecord<Double>(1.4d, initialTime + 4));
-		testHarness.processElement1(new StreamRecord<Double>(1.5d, initialTime + 5));
-
-		testHarness.processElement2(new StreamRecord<Integer>(1, initialTime + 1));
-		testHarness.processElement2(new StreamRecord<Integer>(2, initialTime + 2));
-		testHarness.processWatermark2(new Watermark(initialTime + 2));
-		testHarness.processElement2(new StreamRecord<Integer>(3, initialTime + 3));
-		testHarness.processElement2(new StreamRecord<Integer>(4, initialTime + 4));
-		testHarness.processElement2(new StreamRecord<Integer>(5, initialTime + 5));
-
-		expectedOutput.add(new StreamRecord<String>("1.1", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("1.2", initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("1.3", initialTime + 3));
-		expectedOutput.add(new StreamRecord<String>("1.4", initialTime + 4));
-		expectedOutput.add(new StreamRecord<String>("1.5", initialTime + 5));
-
-		expectedOutput.add(new StreamRecord<String>("1", initialTime + 1));
-		expectedOutput.add(new StreamRecord<String>("2", initialTime + 2));
-		expectedOutput.add(new Watermark(initialTime + 2));
-		expectedOutput.add(new StreamRecord<String>("3", initialTime + 3));
-		expectedOutput.add(new StreamRecord<String>("4", initialTime + 4));
-		expectedOutput.add(new StreamRecord<String>("5", initialTime + 5));
-
-		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
-	}
-
-	@Test
-	public void testOpenClose() throws Exception {
-		CoStreamMap<Double, Integer, String> operator = new CoStreamMap<Double, Integer, String>(new TestOpenCloseCoMapFunction());
-
-		TwoInputStreamOperatorTestHarness<Double, Integer, String> testHarness = new TwoInputStreamOperatorTestHarness<Double, Integer, String>(operator);
-
-		long initialTime = 0L;
-
-		testHarness.open();
-
-		testHarness.processElement1(new StreamRecord<Double>(74d, initialTime));
-		testHarness.processElement2(new StreamRecord<Integer>(42, initialTime));
-
-		testHarness.close();
-
-		Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseCoMapFunction.closeCalled);
-		Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
-	}
-
-	// This must only be used in one test, otherwise the static fields will be changed
-	// by several tests concurrently
-	private static class TestOpenCloseCoMapFunction extends RichCoMapFunction<Double, Integer, String> {
-		private static final long serialVersionUID = 1L;
-
-		public static boolean openCalled = false;
-		public static boolean closeCalled = false;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			if (closeCalled) {
-				Assert.fail("Close called before open.");
-			}
-			openCalled = true;
-		}
-
-		@Override
-		public void close() throws Exception {
-			super.close();
-			if (!openCalled) {
-				Assert.fail("Open was not called before close.");
-			}
-			closeCalled = true;
-		}
-
-		@Override
-		public String map1(Double value) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return value.toString();
-		}
-
-		@Override
-		public String map2(Integer value) throws Exception {
-			if (!openCalled) {
-				Assert.fail("Open was not called before run.");
-			}
-			return value.toString();
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java
deleted file mode 100644
index 130842e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoWindowTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-///*
-// * Licensed to the Apache Software Foundation (ASF) under one or more
-// * contributor license agreements.  See the NOTICE file distributed with
-// * this work for additional information regarding copyright ownership.
-// * The ASF licenses this file to You under the Apache License, Version 2.0
-// * (the "License"); you may not use this file except in compliance with
-// * the License.  You may obtain a copy of the License at
-// *
-// *    http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing, software
-// * distributed under the License is distributed on an "AS IS" BASIS,
-// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// * See the License for the specific language governing permissions and
-// * limitations under the License.
-// */
-//
-//package org.apache.flink.streaming.api.operators.co;
-//
-//import static org.junit.Assert.assertEquals;
-//
-//import java.util.ArrayList;
-//import java.util.HashSet;
-//import java.util.List;
-//import java.util.Set;
-//
-//import org.apache.flink.api.java.tuple.Tuple2;
-//import org.apache.flink.streaming.api.functions.co.CoWindowFunction;
-//import org.apache.flink.streaming.api.operators.co.CoStreamWindow;
-//import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-//import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-//import org.apache.flink.streaming.util.MockCoContext;
-//import org.apache.flink.util.Collector;
-//import org.junit.Test;
-//
-//public class CoWindowTest {
-//
-//	public static final class MyCoGroup1 implements CoWindowFunction<Integer, Integer, Integer> {
-//
-//		private static final long serialVersionUID = 1L;
-//
-//		@SuppressWarnings("unused")
-//		@Override
-//		public void coWindow(List<Integer> first, List<Integer> second, Collector<Integer> out)
-//				throws Exception {
-//			Integer count1 = 0;
-//			for (Integer i : first) {
-//				count1++;
-//			}
-//			Integer count2 = 0;
-//			for (Integer i : second) {
-//				count2++;
-//			}
-//			out.collect(count1);
-//			out.collect(count2);
-//
-//		}
-//
-//	}
-//
-//	public static final class MyCoGroup2 implements
-//			CoWindowFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer> {
-//
-//		private static final long serialVersionUID = 1L;
-//
-//		@Override
-//		public void coWindow(List<Tuple2<Integer, Integer>> first,
-//				List<Tuple2<Integer, Integer>> second, Collector<Integer> out) throws Exception {
-//
-//			Set<Integer> firstElements = new HashSet<Integer>();
-//			for (Tuple2<Integer, Integer> value : first) {
-//				firstElements.add(value.f1);
-//			}
-//			for (Tuple2<Integer, Integer> value : second) {
-//				if (firstElements.contains(value.f1)) {
-//					out.collect(value.f1);
-//				}
-//			}
-//
-//		}
-//
-//	}
-//
-//	private static final class MyTS1 implements Timestamp<Integer> {
-//
-//		private static final long serialVersionUID = 1L;
-//
-//		@Override
-//		public long getTimestamp(Integer value) {
-//			return value;
-//		}
-//
-//	}
-//
-//	private static final class MyTS2 implements Timestamp<Tuple2<Integer, Integer>> {
-//
-//		private static final long serialVersionUID = 1L;
-//
-//		@Override
-//		public long getTimestamp(Tuple2<Integer, Integer> value) {
-//			return value.f0;
-//		}
-//
-//	}
-//
-//	@Test
-//	public void coWindowGroupReduceTest2() throws Exception {
-//
-//		CoStreamWindow<Integer, Integer, Integer> invokable1 = new CoStreamWindow<Integer, Integer, Integer>(
-//				new MyCoGroup1(), 2, 1, new TimestampWrapper<Integer>(new MyTS1(), 1),
-//				new TimestampWrapper<Integer>(new MyTS1(), 1));
-//
-//		// Windowsize 2, slide 1
-//		// 1,2|2,3|3,4|4,5
-//
-//		List<Integer> input11 = new ArrayList<Integer>();
-//		input11.add(1);
-//		input11.add(1);
-//		input11.add(2);
-//		input11.add(3);
-//		input11.add(3);
-//
-//		List<Integer> input12 = new ArrayList<Integer>();
-//		input12.add(1);
-//		input12.add(2);
-//		input12.add(3);
-//		input12.add(3);
-//		input12.add(5);
-//
-//		// Windows: (1,1,2)(1,1,2)|(2,3,3)(2,3,3)|(3,3)(3,3)|(5)(5)
-//		// expected output: 3,2|3,3|2,2|0,1
-//
-//		List<Integer> expected1 = new ArrayList<Integer>();
-//		expected1.add(3);
-//		expected1.add(2);
-//		expected1.add(3);
-//		expected1.add(3);
-//		expected1.add(2);
-//		expected1.add(2);
-//		expected1.add(0);
-//		expected1.add(1);
-//
-//		List<Integer> actual1 = MockCoContext.createAndExecute(invokable1, input11, input12);
-//		assertEquals(expected1, actual1);
-//
-//		CoStreamWindow<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer> invokable2 = new CoStreamWindow<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer>(
-//				new MyCoGroup2(), 2, 3, new TimestampWrapper<Tuple2<Integer, Integer>>(new MyTS2(),
-//						1), new TimestampWrapper<Tuple2<Integer, Integer>>(new MyTS2(), 1));
-//
-//		// WindowSize 2, slide 3
-//		// 1,2|4,5|7,8|
-//
-//		List<Tuple2<Integer, Integer>> input21 = new ArrayList<Tuple2<Integer, Integer>>();
-//		input21.add(new Tuple2<Integer, Integer>(1, 1));
-//		input21.add(new Tuple2<Integer, Integer>(1, 2));
-//		input21.add(new Tuple2<Integer, Integer>(2, 3));
-//		input21.add(new Tuple2<Integer, Integer>(3, 4));
-//		input21.add(new Tuple2<Integer, Integer>(3, 5));
-//		input21.add(new Tuple2<Integer, Integer>(4, 6));
-//		input21.add(new Tuple2<Integer, Integer>(4, 7));
-//		input21.add(new Tuple2<Integer, Integer>(5, 8));
-//
-//		List<Tuple2<Integer, Integer>> input22 = new ArrayList<Tuple2<Integer, Integer>>();
-//		input22.add(new Tuple2<Integer, Integer>(1, 1));
-//		input22.add(new Tuple2<Integer, Integer>(2, 0));
-//		input22.add(new Tuple2<Integer, Integer>(2, 2));
-//		input22.add(new Tuple2<Integer, Integer>(3, 9));
-//		input22.add(new Tuple2<Integer, Integer>(3, 4));
-//		input22.add(new Tuple2<Integer, Integer>(4, 10));
-//		input22.add(new Tuple2<Integer, Integer>(5, 8));
-//		input22.add(new Tuple2<Integer, Integer>(5, 7));
-//
-//		List<Integer> expected2 = new ArrayList<Integer>();
-//		expected2.add(1);
-//		expected2.add(2);
-//		expected2.add(8);
-//		expected2.add(7);
-//
-//		List<Integer> actual2 = MockCoContext.createAndExecute(invokable2, input21, input22);
-//		assertEquals(expected2, actual2);
-//	}
-//}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
deleted file mode 100644
index d00dc67..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.co;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
-
-	private static List<String> expected;
-
-	/**
-	 * We connect two different data streams in a chain to a CoMap.
-	 */
-	@Test
-	public void differentDataStreamSameChain() {
-
-		TestListResultSink<String> resultSink = new TestListResultSink<String>();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		DataStream<Integer> src = env.fromElements(1, 3, 5);
-
-		DataStream<String> stringMap = src.map(new MapFunction<Integer, String>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public String map(Integer value) throws Exception {
-				return "x " + value;
-			}
-		});
-
-		stringMap.connect(src).map(new CoMapFunction<String, Integer, String>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public String map1(String value) {
-				return value;
-			}
-
-			@Override
-			public String map2(Integer value) {
-				return String.valueOf(value + 1);
-			}
-		}).addSink(resultSink);
-
-		try {
-			env.execute();
-		} catch (Exception e) {
-			e.printStackTrace();
-		}
-
-		expected = new ArrayList<String>();
-
-		expected.addAll(Arrays.asList("x 1", "x 3", "x 5", "2", "4", "6"));
-
-		List<String> result = resultSink.getResult();
-
-		Collections.sort(expected);
-		Collections.sort(result);
-
-		assertEquals(expected, result);
-	}
-
-	/**
-	 * We connect two different data streams in different chains to a CoMap.
-	 * (This is not actually self-connect.)
-	 */
-	@Test
-	public void differentDataStreamDifferentChain() {
-
-		TestListResultSink<String> resultSink = new TestListResultSink<String>();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataStream<Integer> src = env.fromElements(1, 3, 5).disableChaining();
-
-		DataStream<String> stringMap = src.flatMap(new FlatMapFunction<Integer, String>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void flatMap(Integer value, Collector<String> out) throws Exception {
-				out.collect("x " + value);
-			}
-		}).keyBy(new KeySelector<String, Integer>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Integer getKey(String value) throws Exception {
-				return value.length();
-			}
-		});
-
-		DataStream<Long> longMap = src.map(new MapFunction<Integer, Long>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Long map(Integer value) throws Exception {
-				return Long.valueOf(value + 1);
-			}
-		}).keyBy(new KeySelector<Long, Long>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Long getKey(Long value) throws Exception {
-				return value;
-			}
-		});
-
-
-		stringMap.connect(longMap).map(new CoMapFunction<String, Long, String>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public String map1(String value) {
-				return value;
-			}
-
-			@Override
-			public String map2(Long value) {
-				return value.toString();
-			}
-		}).addSink(resultSink);
-
-		try {
-			env.execute();
-		} catch (Exception e) {
-			e.printStackTrace();
-		}
-
-		expected = new ArrayList<String>();
-
-		expected.addAll(Arrays.asList("x 1", "x 3", "x 5", "2", "4", "6"));
-
-		List<String> result = resultSink.getResult();
-
-		Collections.sort(expected);
-		Collections.sort(result);
-
-		assertEquals(expected, result);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java
deleted file mode 100644
index 5377e09..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/CsvOutputFormatITCase.java
+++ /dev/null
@@ -1,78 +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.outputformat;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.util.Collector;
-
-public class CsvOutputFormatITCase extends StreamingProgramTestBase {
-
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<String> text = env.fromElements(WordCountData.TEXT);
-
-		DataStream<Tuple2<String, Integer>> counts = text
-				.flatMap(new Tokenizer())
-				.keyBy(0).sum(1);
-
-		counts.writeAsCsv(resultPath);
-
-		env.execute("WriteAsCsvTest");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		//Strip the parentheses from the expected text like output
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES
-				.replaceAll("[\\\\(\\\\)]", ""), resultPath);
-	}
-
-	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<Tuple2<String, Integer>> out)
-				throws Exception {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Tuple2<String, Integer>(token, 1));
-				}
-			}
-		}
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java
deleted file mode 100644
index 49876ec..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/SocketOutputFormatITCase.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.outputformat;
-
-import org.apache.flink.api.common.functions.MapFunction;
-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.util.SocketOutputTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-import org.junit.Ignore;
-
-@Ignore
-//This test sometimes failes most likely due to the behaviour
-//of the socket. Disabled for now.
-public class SocketOutputFormatITCase extends SocketOutputTestBase {
-
-	@Override
-	protected void testProgram() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<String> text = env.fromElements(WordCountData.TEXT);
-
-		DataStream<String> counts =
-				text.flatMap(new CsvOutputFormatITCase.Tokenizer())
-						.keyBy(0).sum(1).map(new MapFunction<Tuple2<String, Integer>, String>() {
-					@Override
-					public String map(Tuple2<String, Integer> value) throws Exception {
-						return value.toString() + "\n";
-					}
-				});
-		counts.writeToSocket(HOST, port, new DummyStringSchema());
-
-		env.execute("WriteToSocketTest");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java
deleted file mode 100644
index 380f00d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/outputformat/TextOutputFormatITCase.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.outputformat;
-
-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.util.StreamingProgramTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class TextOutputFormatITCase extends StreamingProgramTestBase {
-
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<String> text = env.fromElements(WordCountData.TEXT);
-
-		DataStream<Tuple2<String, Integer>> counts = text
-				.flatMap(new CsvOutputFormatITCase.Tokenizer())
-				.keyBy(0).sum(1);
-
-		counts.writeAsText(resultPath);
-
-		env.execute("WriteAsTextTest");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java
deleted file mode 100644
index 317a21c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/MockRecordWriter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.streamtask;
-
-import java.util.ArrayList;
-
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.runtime.operators.DataSourceTask;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
-
-public class MockRecordWriter extends RecordWriter<SerializationDelegate<StreamRecord<Tuple1<Integer>>>> {
-
-	public ArrayList<Integer> emittedRecords;
-
-	public MockRecordWriter(DataSourceTask<?> inputBase, Class<StreamRecord<Tuple1<Integer>>> outputClass) {
-		super(inputBase.getEnvironment().getWriter(0));
-	}
-
-	public boolean initList() {
-		emittedRecords = new ArrayList<Integer>();
-		return true;
-	}
-	
-	@Override
-	public void emit(SerializationDelegate<StreamRecord<Tuple1<Integer>>> record) {
-		emittedRecords.add(record.getInstance().getValue().f0);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java
deleted file mode 100644
index 8f5f8df..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamIterationHeadTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.streamtask;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.tasks.StreamIterationHead;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskTestHarness;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import static org.junit.Assert.*;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ ResultPartitionWriter.class })
-public class StreamIterationHeadTest {
-
-	@Test
-	public void testIterationHeadWatermarkEmission() throws Exception {
-		StreamIterationHead<Integer> head = new StreamIterationHead<>();
-		StreamTaskTestHarness<Integer> harness = new StreamTaskTestHarness<>(head,
-				BasicTypeInfo.INT_TYPE_INFO);
-		harness.getStreamConfig().setIterationId("1");
-		harness.getStreamConfig().setIterationWaitTime(1);
-
-		harness.invoke();
-		harness.waitForTaskCompletion();
-
-		assertEquals(1, harness.getOutput().size());
-		assertEquals(new Watermark(Long.MAX_VALUE), harness.getOutput().peek());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java
deleted file mode 100644
index 122aa8a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamtask/StreamVertexTest.java
+++ /dev/null
@@ -1,187 +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.streamtask;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.junit.Test;
-
-public class StreamVertexTest extends StreamingMultipleProgramsTestBase {
-
-	private static Map<Integer, Integer> data = new HashMap<Integer, Integer>();
-
-	public static class MySource implements SourceFunction<Tuple1<Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		private Tuple1<Integer> tuple = new Tuple1<Integer>(0);
-
-		private int i = 0;
-
-		@Override
-		public void run(SourceContext<Tuple1<Integer>> ctx) throws Exception {
-			for (int i = 0; i < 10; i++) {
-				tuple.f0 = i;
-				ctx.collect(tuple);
-			}
-		}
-
-		@Override
-		public void cancel() {
-		}
-	}
-
-	public static class MyTask extends RichMapFunction<Tuple1<Integer>, Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> map(Tuple1<Integer> value) throws Exception {
-			Integer i = value.f0;
-			return new Tuple2<Integer, Integer>(i, i + 1);
-		}
-	}
-
-	public static class MySink implements SinkFunction<Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void invoke(Tuple2<Integer, Integer> tuple) {
-			Integer k = tuple.getField(0);
-			Integer v = tuple.getField(1);
-			data.put(k, v);
-		}
-
-	}
-
-	@SuppressWarnings("unused")
-	private static final int SOURCE_PARALELISM = 1;
-
-	@Test
-	public void wrongJobGraph() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(SOURCE_PARALELISM);
-
-		try {
-			env.fromCollection(null);
-			fail();
-		} catch (NullPointerException e) {
-		}
-
-		try {
-			env.fromElements();
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-
-		try {
-			env.generateSequence(-10, -30);
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-
-		try {
-			env.setBufferTimeout(-10);
-			fail();
-		} catch (IllegalArgumentException e) {
-		}
-
-		try {
-			env.generateSequence(1, 10).project(2);
-			fail();
-		} catch (RuntimeException e) {
-		}
-	}
-
-	private static class CoMap implements CoMapFunction<String, Long, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map1(String value) {
-			// System.out.println(value);
-			return value;
-		}
-
-		@Override
-		public String map2(Long value) {
-			// System.out.println(value);
-			return value.toString();
-		}
-	}
-
-	private static class SetSink implements SinkFunction<String> {
-		private static final long serialVersionUID = 1L;
-		public static Set<String> result = Collections.synchronizedSet(new HashSet<String>());
-
-		@Override
-		public void invoke(String value) {
-			result.add(value);
-		}
-
-	}
-
-	@Test
-	public void coTest() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(SOURCE_PARALELISM);
-
-		DataStream<String> fromStringElements = env.fromElements("aa", "bb", "cc");
-		DataStream<Long> generatedSequence = env.generateSequence(0, 3);
-
-		fromStringElements.connect(generatedSequence).map(new CoMap()).addSink(new SetSink());
-
-		env.execute();
-
-		HashSet<String> expectedSet = new HashSet<String>(Arrays.asList("aa", "bb", "cc", "0", "1",
-				"2", "3"));
-		assertEquals(expectedSet, SetSink.result);
-	}
-
-	@Test
-	public void runStream() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(SOURCE_PARALELISM);
-
-		env.addSource(new MySource()).setParallelism(SOURCE_PARALELISM).map(new MyTask())
-				.addSink(new MySink());
-
-		env.execute();
-		assertEquals(10, data.keySet().size());
-
-		for (Integer k : data.keySet()) {
-			assertEquals((Integer) (k + 1), data.get(k));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/CosineDistanceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/CosineDistanceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/CosineDistanceTest.java
deleted file mode 100644
index bdc7e94..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/CosineDistanceTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.windowing.deltafunction;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.streaming.api.functions.windowing.delta.CosineDistance;
-import org.junit.Test;
-
-public class CosineDistanceTest {
-	
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testCosineDistance() {
-		
-		//Reference calculated using wolfram alpha
-		double[][][] testdata={
-				{{0,0,0},{0,0,0}},
-				{{0,0,0},{1,2,3}},
-				{{1,2,3},{0,0,0}},
-				{{1,2,3},{4,5,6}},
-				{{1,2,3},{-4,-5,-6}},
-				{{1,2,-3},{-4,5,-6}},
-				{{1,2,3,4},{5,6,7,8}},
-				{{1,2},{3,4}},
-				{{1},{2}},
-			};
-		double[] referenceSolutions={
-				0,
-				0,
-				0,
-				0.025368,
-				1.974631,
-				0.269026,
-				0.031136,
-				0.016130,
-				0
-		};
-		
-		for (int i = 0; i < testdata.length; i++) {
-			assertEquals("Wrong result for inputs " + arrayToString(testdata[i][0]) + " and "
-					+ arrayToString(testdata[i][0]), referenceSolutions[i],
-					new CosineDistance().getDelta(testdata[i][0], testdata[i][1]), 0.000001);
-		}
-	}
-	
-	private String arrayToString(double[] in){
-		if (in.length==0) return "{}";
-		String result="{";
-		for (double d:in){
-			result+=d+",";
-		}
-		return result.substring(0, result.length()-1)+"}";
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/EuclideanDistanceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/EuclideanDistanceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/EuclideanDistanceTest.java
deleted file mode 100644
index 85a0882..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/deltafunction/EuclideanDistanceTest.java
+++ /dev/null
@@ -1,72 +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.deltafunction;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.streaming.api.functions.windowing.delta.EuclideanDistance;
-import org.junit.Test;
-
-public class EuclideanDistanceTest {
-	
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testEuclideanDistance() {
-		
-		//Reference calculated using wolfram alpha
-		double[][][] testdata={
-				{{0,0,0},{0,0,0}},
-				{{0,0,0},{1,2,3}},
-				{{1,2,3},{0,0,0}},
-				{{1,2,3},{4,5,6}},
-				{{1,2,3},{-4,-5,-6}},
-				{{1,2,-3},{-4,5,-6}},
-				{{1,2,3,4},{5,6,7,8}},
-				{{1,2},{3,4}},
-				{{1},{2}},
-			};
-		double[] referenceSolutions={
-				0,
-				3.741657,
-				3.741657,
-				5.196152,
-				12.4499,
-				6.557439,
-				8.0,
-				2.828427,
-				1
-		};
-		
-		for (int i = 0; i < testdata.length; i++) {
-			assertEquals("Wrong result for inputs " + arrayToString(testdata[i][0]) + " and "
-					+ arrayToString(testdata[i][0]), referenceSolutions[i],
-					new EuclideanDistance().getDelta(testdata[i][0], testdata[i][1]), 0.000001);
-		}
-		
-	}
-	
-	private String arrayToString(double[] in){
-		if (in.length==0) return "{}";
-		String result="{";
-		for (double d:in){
-			result+=d+",";
-		}
-		return result.substring(0, result.length()-1)+"}";
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java
deleted file mode 100644
index 9d9d47b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/graph/TranslationTest.java
+++ /dev/null
@@ -1,76 +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.graph;
-
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-@SuppressWarnings("serial")
-public class TranslationTest extends StreamingMultipleProgramsTestBase {
-	
-	@Test
-	public void testCheckpointModeTranslation() {
-		try {
-			// with deactivated fault tolerance, the checkpoint mode should be at-least-once
-			StreamExecutionEnvironment deactivated = getSimpleJob();
-			
-			for (JobVertex vertex : deactivated.getStreamGraph().getJobGraph().getVertices()) {
-				assertEquals(CheckpointingMode.AT_LEAST_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode());
-			}
-
-			// with activated fault tolerance, the checkpoint mode should be by default exactly once
-			StreamExecutionEnvironment activated = getSimpleJob();
-			activated.enableCheckpointing(1000L);
-			for (JobVertex vertex : activated.getStreamGraph().getJobGraph().getVertices()) {
-				assertEquals(CheckpointingMode.EXACTLY_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode());
-			}
-
-			// explicitly setting the mode
-			StreamExecutionEnvironment explicit = getSimpleJob();
-			explicit.enableCheckpointing(1000L, CheckpointingMode.AT_LEAST_ONCE);
-			for (JobVertex vertex : explicit.getStreamGraph().getJobGraph().getVertices()) {
-				assertEquals(CheckpointingMode.AT_LEAST_ONCE, new StreamConfig(vertex.getConfiguration()).getCheckpointMode());
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	private static StreamExecutionEnvironment getSimpleJob() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.generateSequence(1, 10000000)
-				.addSink(new SinkFunction<Long>() {
-					@Override
-					public void invoke(Long value) {
-					}
-				});
-		
-		return env;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
deleted file mode 100644
index a8a989b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.io;
-
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.Random;
-
-import org.apache.flink.core.memory.MemoryType;
-import org.apache.flink.runtime.event.TaskEvent;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferPool;
-import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
-import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-
-import org.junit.Test;
-
-/**
- * The test generates two random streams (input channels) which independently
- * and randomly generate checkpoint barriers. The two streams are very
- * unaligned, putting heavy work on the BarrierBuffer.
- */
-public class BarrierBufferMassiveRandomTest {
-
-	private static final int PAGE_SIZE = 1024;
-	
-	@Test
-	public void testWithTwoChannelsAndRandomBarriers() {
-		IOManager ioMan = null;
-		try {
-			ioMan = new IOManagerAsync();
-			
-			BufferPool pool1 = new NetworkBufferPool(100, PAGE_SIZE, MemoryType.HEAP).createBufferPool(100, true);
-			BufferPool pool2 = new NetworkBufferPool(100, PAGE_SIZE, MemoryType.HEAP).createBufferPool(100, true);
-
-			RandomGeneratingInputGate myIG = new RandomGeneratingInputGate(
-					new BufferPool[] { pool1, pool2 },
-					new BarrierGenerator[] { new CountBarrier(100000), new RandomBarrier(100000) });
-	
-			BarrierBuffer barrierBuffer = new BarrierBuffer(myIG, ioMan);
-			
-			for (int i = 0; i < 2000000; i++) {
-				BufferOrEvent boe = barrierBuffer.getNextNonBlocked();
-				if (boe.isBuffer()) {
-					boe.getBuffer().recycle();
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-		finally {
-			if (ioMan != null) {
-				ioMan.shutdown();
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Mocks and Generators
-	// ------------------------------------------------------------------------
-	
-	protected interface BarrierGenerator {
-		public boolean isNextBarrier();
-	}
-
-	protected static class RandomBarrier implements BarrierGenerator {
-		
-		private static final Random rnd = new Random();
-
-		private final double threshold;
-
-		public RandomBarrier(double expectedEvery) {
-			threshold = 1 / expectedEvery;
-		}
-
-		@Override
-		public boolean isNextBarrier() {
-			return rnd.nextDouble() < threshold;
-		}
-	}
-
-	private static class CountBarrier implements BarrierGenerator {
-
-		private final long every;
-		private long c = 0;
-
-		public CountBarrier(long every) {
-			this.every = every;
-		}
-
-		@Override
-		public boolean isNextBarrier() {
-			return c++ % every == 0;
-		}
-	}
-
-	protected static class RandomGeneratingInputGate implements InputGate {
-
-		private final int numChannels;
-		private final BufferPool[] bufferPools;
-		private final int[] currentBarriers;
-		private final BarrierGenerator[] barrierGens;
-		private int currentChannel = 0;
-		private long c = 0;
-
-		public RandomGeneratingInputGate(BufferPool[] bufferPools, BarrierGenerator[] barrierGens) {
-			this.numChannels = bufferPools.length;
-			this.currentBarriers = new int[numChannels];
-			this.bufferPools = bufferPools;
-			this.barrierGens = barrierGens;
-		}
-
-		@Override
-		public int getNumberOfInputChannels() {
-			return numChannels;
-		}
-
-		@Override
-		public boolean isFinished() {
-			return false;
-		}
-
-		@Override
-		public void requestPartitions() {}
-
-		@Override
-		public BufferOrEvent getNextBufferOrEvent() throws IOException, InterruptedException {
-			currentChannel = (currentChannel + 1) % numChannels;
-
-			if (barrierGens[currentChannel].isNextBarrier()) {
-				return new BufferOrEvent(
-						new CheckpointBarrier(++currentBarriers[currentChannel], System.currentTimeMillis()),
-							currentChannel);
-			} else {
-				Buffer buffer = bufferPools[currentChannel].requestBuffer();
-				buffer.getMemorySegment().putLong(0, c++);
-				return new BufferOrEvent(buffer, currentChannel);
-			}
-		}
-
-		@Override
-		public void sendTaskEvent(TaskEvent event) {}
-
-		@Override
-		public void registerListener(EventListener<InputGate> listener) {}
-
-		@Override
-		public int getPageSize() {
-			return PAGE_SIZE;
-		}
-	}
-}


[16/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
deleted file mode 100644
index 08ce890..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleScalaPrograms.join;
-
-import org.apache.flink.streaming.scala.examples.join.WindowJoin;
-import org.apache.flink.streaming.examples.join.util.WindowJoinData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class WindowJoinITCase extends StreamingProgramTestBase {
-
-	protected String gradesPath;
-	protected String salariesPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
-		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		// since the two sides of the join might have different speed
-		// the exact output can not be checked just whether it is well-formed
-		// checks that the result lines look like e.g. Person(bob, 2, 2015)
-		checkLinesAgainstRegexp(resultPath, "^Person\\([a-z]+,(\\d),(\\d)+\\)");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java
deleted file mode 100644
index b3629ad..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java
+++ /dev/null
@@ -1,30 +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.test.exampleScalaPrograms.socket;
-
-import org.apache.flink.streaming.scala.examples.socket.SocketTextStreamWordCount;
-import org.apache.flink.streaming.util.SocketProgramITCaseBase;
-
-public class SocketTextStreamWordCountITCase extends SocketProgramITCaseBase {
-
-	@Override
-	protected void testProgram() throws Exception {
-		SocketTextStreamWordCount.main(new String[]{HOST, port.toString(), resultPath});
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java
deleted file mode 100644
index ef4e47f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleScalaPrograms.windowing;
-
-import org.apache.flink.streaming.scala.examples.windowing.TopSpeedWindowing;
-import org.apache.flink.streaming.examples.windowing.util.TopSpeedWindowingExampleData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class TopSpeedWindowingExampleITCase extends StreamingProgramTestBase {
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		setParallelism(1); //needed to ensure total ordering for windows
-		textPath = createTempFile("text.txt", TopSpeedWindowingExampleData.CAR_DATA);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(TopSpeedWindowingExampleData.TOP_CASE_CLASS_SPEEDS, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		TopSpeedWindowing.main(new String[]{textPath, resultPath});
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/pom.xml b/flink-staging/flink-streaming/flink-streaming-scala/pom.xml
deleted file mode 100644
index aa0d67e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/pom.xml
+++ /dev/null
@@ -1,236 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-streaming-scala</artifactId>
-	<name>flink-streaming-scala</name>
-	<packaging>jar</packaging>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-scala</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.scala-lang</groupId>
-			<artifactId>scala-reflect</artifactId>
-		</dependency>
-
-		<dependency>
-			<groupId>org.scala-lang</groupId>
-			<artifactId>scala-library</artifactId>
-		</dependency>
-
-		<dependency>
-			<groupId>org.scala-lang</groupId>
-			<artifactId>scala-compiler</artifactId>
-		</dependency>
-
-		<dependency>
-			<groupId>org.ow2.asm</groupId>
-			<artifactId>asm</artifactId>
-			<version>${asm.version}</version>
-		</dependency>
-		
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<!-- To access general test utils -->
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<!-- To access test data -->
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<!-- To access streaming test utils -->
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- Scala Compiler -->
-			<plugin>
-				<groupId>net.alchim31.maven</groupId>
-				<artifactId>scala-maven-plugin</artifactId>
-				<version>3.1.4</version>
-				<executions>
-					<!-- Run scala compiler in the process-resources phase, so that dependencies on
-						scala classes can be resolved later in the (Java) compile phase -->
-					<execution>
-						<id>scala-compile-first</id>
-						<phase>process-resources</phase>
-						<goals>
-							<goal>compile</goal>
-						</goals>
-					</execution>
- 
-					<!-- Run scala compiler in the process-test-resources phase, so that dependencies on
-						 scala classes can be resolved later in the (Java) test-compile phase -->
-					<execution>
-						<id>scala-test-compile</id>
-						<phase>process-test-resources</phase>
-						<goals>
-							<goal>testCompile</goal>
-						</goals>
-					</execution>
-				</executions>
-				<configuration>
-					<jvmArgs>
-						<jvmArg>-Xms128m</jvmArg>
-						<jvmArg>-Xmx512m</jvmArg>
-					</jvmArgs>
-					<compilerPlugins combine.children="append">
-					   <compilerPlugin>
-						   <groupId>org.scalamacros</groupId>
-						   <artifactId>paradise_${scala.version}</artifactId>
-						   <version>${scala.macros.version}</version>
-					   </compilerPlugin>
-				   </compilerPlugins>
-				</configuration>
-			</plugin>
-
-			<!-- Eclipse Integration -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-eclipse-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<downloadSources>true</downloadSources>
-					<projectnatures>
-						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
-						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
-					</projectnatures>
-					<buildcommands>
-						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
-					</buildcommands>
-					<classpathContainers>
-						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
-						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
-					</classpathContainers>
-					<excludes>
-						<exclude>org.scala-lang:scala-library</exclude>
-						<exclude>org.scala-lang:scala-compiler</exclude>
-					</excludes>
-					<sourceIncludes>
-						<sourceInclude>**/*.scala</sourceInclude>
-						<sourceInclude>**/*.java</sourceInclude>
-					</sourceIncludes>
-				</configuration>
-			</plugin>
-
-			<!-- Adding scala source directories to build path -->
-			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>build-helper-maven-plugin</artifactId>
-				<version>1.7</version>
-				<executions>
-					<!-- Add src/main/scala to eclipse build path -->
-					<execution>
-						<id>add-source</id>
-						<phase>generate-sources</phase>
-						<goals>
-							<goal>add-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/main/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-					<!-- Add src/test/scala to eclipse build path -->
-					<execution>
-						<id>add-test-source</id>
-						<phase>generate-test-sources</phase>
-						<goals>
-							<goal>add-test-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/test/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<plugin>
-				<groupId>org.scalastyle</groupId>
-				<artifactId>scalastyle-maven-plugin</artifactId>
-				<version>0.5.0</version>
-				<executions>
-					<execution>
-						<goals>
-							<goal>check</goal>
-						</goals>
-					</execution>
-				</executions>
-				<configuration>
-					<verbose>false</verbose>
-					<failOnViolation>true</failOnViolation>
-					<includeTestSourceDirectory>true</includeTestSourceDirectory>
-					<failOnWarning>false</failOnWarning>
-					<sourceDirectory>${basedir}/src/main/scala</sourceDirectory>
-					<testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory>
-					<configLocation>${project.basedir}/../../../tools/maven/scalastyle-config.xml</configLocation>
-					<outputFile>${project.basedir}/scalastyle-output.xml</outputFile>
-					<outputEncoding>UTF-8</outputEncoding>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
deleted file mode 100644
index 0357144..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWStream}
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
-import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
-import org.apache.flink.streaming.api.windowing.evictors.Evictor
-import org.apache.flink.streaming.api.windowing.triggers.Trigger
-import org.apache.flink.streaming.api.windowing.windows.Window
-import org.apache.flink.util.Collector
-
-import scala.reflect.ClassTag
-
-import scala.collection.JavaConverters._
-
-/**
- * A [[AllWindowedStream]] represents a data stream where the stream of
- * elements is split into windows based on a
- * [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]. Window emission
- * is triggered based on a [[Trigger]].
- *
- * If an [[Evictor]] is specified it will be
- * used to evict elements from the window after
- * evaluation was triggered by the [[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.
- *
- * Note that the [[AllWindowedStream()]] is purely and API construct, during runtime
- * the [[AllWindowedStream()]] will be collapsed together with the
- * operation over the window into one single operation.
- *
- * @tparam T The type of elements in the stream.
- * @tparam W The type of [[Window]] that the
- *           [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]
- *           assigns the elements to.
- */
-class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
-
-  /**
-   * Sets the [[Trigger]] that should be used to trigger window emission.
-   */
-  def trigger(trigger: Trigger[_ >: T, _ >: W]): AllWindowedStream[T, W] = {
-    javaStream.trigger(trigger)
-    this
-  }
-
-  /**
-   * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
-   *
-   * Note: When using an evictor window performance will degrade significantly, since
-   * pre-aggregation of window results cannot be used.
-   */
-  def evictor(evictor: Evictor[_ >: T, _ >: W]): AllWindowedStream[T, W] = {
-    javaStream.evictor(evictor)
-    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.
-   *
-   * 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.
-   */
-  def reduce(function: ReduceFunction[T]): DataStream[T] = {
-    javaStream.reduce(clean(function))
-  }
-
-  /**
-   * 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.
-   *
-   * 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.
-   */
-  def reduce(function: (T, T) => T): DataStream[T] = {
-    if (function == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    val cleanFun = clean(function)
-    val reducer = new ReduceFunction[T] {
-      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
-    }
-    reduce(reducer)
-  }
-
-  /**
-   * Applies the given fold function to each 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.
-   *
-   * @param function The fold function.
-   * @return The data stream that is the result of applying the fold function to the window.
-   */
-  def fold[R: TypeInformation: ClassTag](
-      initialValue: R,
-      function: FoldFunction[T,R]): DataStream[R] = {
-    if (function == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-
-    val resultType : TypeInformation[R] = implicitly[TypeInformation[R]]
-
-    javaStream.fold(initialValue, function, resultType)
-  }
-
-  /**
-   * Applies the given fold function to each 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.
-   *
-   * @param function The fold function.
-   * @return The data stream that is the result of applying the fold function to the window.
-   */
-  def fold[R: TypeInformation: ClassTag](initialValue: R, function: (R, T) => R): DataStream[R] = {
-    if (function == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    val cleanFun = clean(function)
-    val folder = new FoldFunction[T,R] {
-      def fold(acc: R, v: T) = {
-        cleanFun(acc, v)
-      }
-    }
-    fold(initialValue, folder)
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Not that this function requires that all data in the windows is buffered until the window
-   * is evaluated, as the function provides no means of pre-aggregation.
-   *
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](function: AllWindowFunction[T, R, W]): DataStream[R] = {
-    javaStream.apply(clean(function), implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Not that this function requires that all data in the windows is buffered until the window
-   * is evaluated, as the function provides no means of pre-aggregation.
-   *
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](
-      function: (W, Iterable[T], Collector[R]) => Unit): DataStream[R] = {
-    val cleanedFunction = clean(function)
-    val applyFunction = new AllWindowFunction[T, R, W] {
-      def apply(window: W, elements: java.lang.Iterable[T], out: Collector[R]): Unit = {
-        cleanedFunction(window, elements.asScala, out)
-      }
-    }
-    javaStream.apply(applyFunction, implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-   *
-   * @param preAggregator The reduce function that is used for pre-aggregation
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](
-      preAggregator: ReduceFunction[T],
-      function: AllWindowFunction[T, R, W]): DataStream[R] = {
-    javaStream.apply(clean(preAggregator), clean(function), implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-   *
-   * @param preAggregator The reduce function that is used for pre-aggregation
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](
-      preAggregator: (T, T) => T,
-      function: (W, Iterable[T], Collector[R]) => Unit): DataStream[R] = {
-    if (function == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    if (function == null) {
-      throw new NullPointerException("WindowApply function must not be null.")
-    }
-
-    val cleanReducer = clean(preAggregator)
-    val reducer = new ReduceFunction[T] {
-      def reduce(v1: T, v2: T) = { cleanReducer(v1, v2) }
-    }
-
-    val cleanApply = clean(function)
-    val applyFunction = new AllWindowFunction[T, R, W] {
-      def apply(window: W, elements: java.lang.Iterable[T], out: Collector[R]): Unit = {
-        cleanApply(window, elements.asScala, out)
-      }
-    }
-    javaStream.apply(reducer, applyFunction, implicitly[TypeInformation[R]])
-  }
-
-  // ------------------------------------------------------------------------
-  //  Aggregations on the keyed windows
-  // ------------------------------------------------------------------------
-
-  /**
-   * Applies an aggregation that that gives the maximum of the elements in the window at
-   * the given position.
-   */
-  def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
-
-  /**
-   * Applies an aggregation that that gives the maximum of the elements in the window at
-   * the given field.
-   */
-  def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
-
-  /**
-   * Applies an aggregation that that gives the minimum of the elements in the window at
-   * the given position.
-   */
-  def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
-
-  /**
-   * Applies an aggregation that that gives the minimum of the elements in the window at
-   * the given field.
-   */
-  def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
-
-  /**
-   * Applies an aggregation that sums the elements in the window at the given position.
-   */
-  def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
-
-  /**
-   * Applies an aggregation that sums the elements in the window at the given field.
-   */
-  def sum(field: String): DataStream[T] = aggregate(AggregationType.SUM, field)
-
-  /**
-   * Applies an aggregation that that gives the maximum element of the window by
-   * the given position. When equality, returns the first.
-   */
-  def maxBy(position: Int): DataStream[T] = aggregate(AggregationType.MAXBY,
-    position)
-
-  /**
-   * Applies an aggregation that that gives the maximum element of the window by
-   * the given field. When equality, returns the first.
-   */
-  def maxBy(field: String): DataStream[T] = aggregate(AggregationType.MAXBY,
-    field)
-
-  /**
-   * Applies an aggregation that that gives the minimum element of the window by
-   * the given position. When equality, returns the first.
-   */
-  def minBy(position: Int): DataStream[T] = aggregate(AggregationType.MINBY,
-    position)
-
-  /**
-   * Applies an aggregation that that gives the minimum element of the window by
-   * the given field. When equality, returns the first.
-   */
-  def minBy(field: String): DataStream[T] = aggregate(AggregationType.MINBY,
-    field)
-
-  private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
-    val position = fieldNames2Indices(getInputType(), Array(field))(0)
-    aggregate(aggregationType, position)
-  }
-
-  def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
-
-    val jStream = javaStream.asInstanceOf[JavaAllWStream[Product, W]]
-
-    val reducer = aggregationType match {
-      case AggregationType.SUM =>
-        new SumAggregator(position, jStream.getInputType, jStream.getExecutionEnvironment.getConfig)
-
-      case _ =>
-        new ComparableAggregator(
-          position,
-          jStream.getInputType,
-          aggregationType,
-          true,
-          jStream.getExecutionEnvironment.getConfig)
-    }
-
-    new DataStream[Product](jStream.reduce(reducer)).asInstanceOf[DataStream[T]]
-  }
-
-  // ------------------------------------------------------------------------
-  //  Utilities
-  // ------------------------------------------------------------------------
-
-  /**
-   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-   */
-  private[flink] def clean[F <: AnyRef](f: F): F = {
-    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
-  }
-
-  /**
-   * Gets the output type.
-   */
-  private def getInputType(): TypeInformation[T] = javaStream.getInputType
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
deleted file mode 100644
index e676f81..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.functions.CoGroupFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.streaming.api.datastream.{CoGroupedStreams => JavaCoGroupedStreams}
-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.util.Collector
-
-import scala.collection.JavaConverters._
-
-import scala.reflect.ClassTag
-
-/**
- * `CoGroupedStreams` represents two [[DataStream]]s that have been co-grouped.
- * A streaming co-group operation is evaluated over elements in a window.
- *
- * To finalize the co-group operation you also need to specify a [[KeySelector]] for
- * both the first and second input and a [[WindowAssigner]]
- *
- * Note: Right now, the groups are being built in memory so you need to ensure that they don't
- * get too big. Otherwise the JVM might crash.
- *
- * Example:
- *
- * {{{
- * val one: DataStream[(String, Int)]  = ...
- * val two: DataStream[(String, Int)] = ...
- *
- * val result = one.coGroup(two)
- *     .where(new MyFirstKeySelector())
- *     .equalTo(new MyFirstKeySelector())
- *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
- *     .apply(new MyCoGroupFunction())
- * } }}}
- */
-object CoGroupedStreams {
-
-  /**
-   * A co-group operation that does not yet have its [[KeySelector]]s defined.
-   *
-   * @tparam T1 Type of the elements from the first input
-   * @tparam T2 Type of the elements from the second input
-   */
-  class Unspecified[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) {
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the first input.
-     */
-    def where[KEY: TypeInformation](keySelector: T1 => KEY): WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val keyType = implicitly[TypeInformation[KEY]]
-      val javaSelector = new KeySelector[T1, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T1) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = keyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, javaSelector, null, keyType)
-    }
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the second input.
-     */
-    def equalTo[KEY: TypeInformation](keySelector: T2 => KEY): WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val keyType = implicitly[TypeInformation[KEY]]
-      val javaSelector = new KeySelector[T2, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T2) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = keyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, null, javaSelector, keyType)
-    }
-
-    /**
-     * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-     * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-     */
-    private[flink] def clean[F <: AnyRef](f: F): F = {
-      new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
-    }
-  }
-
-  /**
-   * A co-group operation that has [[KeySelector]]s defined for either both or
-   * one input.
-   *
-   * You need to specify a [[KeySelector]] for both inputs using [[where()]] and [[equalTo()]]
-   * before you can proceeed with specifying a [[WindowAssigner]] using [[window()]].
-   *
-   * @tparam T1 Type of the elements from the first input
-   * @tparam T2 Type of the elements from the second input
-   * @tparam KEY Type of the key. This must be the same for both inputs
-   */
-  class WithKey[T1, T2, KEY](
-      input1: DataStream[T1],
-      input2: DataStream[T2],
-      keySelector1: KeySelector[T1, KEY],
-      keySelector2: KeySelector[T2, KEY],
-      keyType: TypeInformation[KEY]) {
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the first input.
-     */
-    def where(keySelector: T1 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val localKeyType = keyType
-      val javaSelector = new KeySelector[T1, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T1) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = localKeyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, javaSelector, keySelector2, keyType)
-    }
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the second input.
-     */
-    def equalTo(keySelector: T2 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val localKeyType = keyType
-      val javaSelector = new KeySelector[T2, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T2) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = localKeyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, keySelector1, javaSelector, keyType)
-    }
-
-    /**
-     * Specifies the window on which the co-group operation works.
-     */
-    def window[W <: Window](
-        assigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W])
-        : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
-      if (keySelector1 == null || keySelector2 == null) {
-        throw new UnsupportedOperationException("You first need to specify KeySelectors for both" +
-          "inputs using where() and equalTo().")
-      }
-      new CoGroupedStreams.WithWindow[T1, T2, KEY, W](
-        input1,
-        input2,
-        keySelector1,
-        keySelector2,
-        clean(assigner),
-        null,
-        null)
-    }
-
-    /**
-     * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-     * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-     */
-    private[flink] def clean[F <: AnyRef](f: F): F = {
-      new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
-    }
-  }
-
-  /**
-   * A co-group operation that has [[KeySelector]]s defined for both inputs as
-   * well as a [[WindowAssigner]].
-   *
-   * @tparam T1 Type of the elements from the first input
-   * @tparam T2 Type of the elements from the second input
-   * @tparam KEY Type of the key. This must be the same for both inputs
-   * @tparam W Type of { @link Window} on which the co-group operation works.
-   */
-  class WithWindow[T1, T2, KEY, W <: Window](
-      input1: DataStream[T1],
-      input2: DataStream[T2],
-      keySelector1: KeySelector[T1, KEY],
-      keySelector2: KeySelector[T2, KEY],
-      windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W],
-      trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W],
-      evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) {
-
-
-    /**
-     * Sets the [[Trigger]] that should be used to trigger window emission.
-     */
-    def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
-    : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
-      new WithWindow[T1, T2, KEY, W](
-        input1,
-        input2,
-        keySelector1,
-        keySelector2,
-        windowAssigner,
-        newTrigger,
-        evictor)
-    }
-
-    /**
-     * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
-     *
-     * Note: When using an evictor window performance will degrade significantly, since
-     * pre-aggregation of window results cannot be used.
-     */
-    def evictor(newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
-    : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
-      new WithWindow[T1, T2, KEY, W](
-        input1,
-        input2,
-        keySelector1,
-        keySelector2,
-        windowAssigner,
-        trigger,
-        newEvictor)
-    }
-
-    /**
-     * Completes the co-group operation with the user function that is executed
-     * for windowed groups.
-     */
-    def apply[O: TypeInformation: ClassTag](
-        fun: (Iterator[T1], Iterator[T2]) => O): DataStream[O] = {
-      require(fun != null, "CoGroup function must not be null.")
-
-      val coGrouper = new CoGroupFunction[T1, T2, O] {
-        val cleanFun = clean(fun)
-        def coGroup(
-            left: java.lang.Iterable[T1],
-            right: java.lang.Iterable[T2], out: Collector[O]) = {
-          out.collect(cleanFun(left.iterator().asScala, right.iterator().asScala))
-        }
-      }
-      apply(coGrouper)
-    }
-
-    /**
-     * Completes the co-group operation with the user function that is executed
-     * for windowed groups.
-     */
-    def apply[O: TypeInformation: ClassTag](
-        fun: (Iterator[T1], Iterator[T2], Collector[O]) => Unit): DataStream[O] = {
-      require(fun != null, "CoGroup function must not be null.")
-
-      val coGrouper = new CoGroupFunction[T1, T2, O] {
-        val cleanFun = clean(fun)
-        def coGroup(
-            left: java.lang.Iterable[T1],
-            right: java.lang.Iterable[T2], out: Collector[O]) = {
-          cleanFun(left.iterator.asScala, right.iterator.asScala, out)
-        }
-      }
-      apply(coGrouper)
-    }
-
-    /**
-     * Completes the co-group operation with the user function that is executed
-     * for windowed groups.
-     */
-    def apply[T: TypeInformation](function: CoGroupFunction[T1, T2, T]): DataStream[T] = {
-
-      val coGroup = new JavaCoGroupedStreams[T1, T2](input1.getJavaStream, input2.getJavaStream)
-
-      coGroup
-        .where(keySelector1)
-        .equalTo(keySelector2)
-        .window(windowAssigner)
-        .trigger(trigger)
-        .evictor(evictor)
-        .apply(clean(function), implicitly[TypeInformation[T]])
-    }
-
-    /**
-     * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-     * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-     */
-    private[flink] def clean[F <: AnyRef](f: F): F = {
-      new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
-    }
-  }
-
-
-  /**
-   * Creates a new co-group operation from the two given inputs.
-   */
-  def createCoGroup[T1, T2](input1: DataStream[T1], input2: DataStream[T2])
-      : CoGroupedStreams.Unspecified[T1, T2] = {
-    new CoGroupedStreams.Unspecified[T1, T2](input1, input2)
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
deleted file mode 100644
index 3ff773f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.api.java.typeutils.{TypeExtractor, ResultTypeQueryable}
-import org.apache.flink.streaming.api.datastream.{ConnectedStreams => JavaCStream, DataStream => JavaStream, KeyedStream => JKeyedStream}
-import org.apache.flink.streaming.api.functions.co.{CoFlatMapFunction, CoMapFunction}
-import org.apache.flink.util.Collector
-import scala.reflect.ClassTag
-
-/**
- * [[ConnectedStreams]] represents two connected streams of (possible) different data types. It
- * can be used to apply transformations such as [[CoMapFunction]] on two
- * [[DataStream]]s.
- */
-class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
-
-  /**
-   * Applies a CoMap transformation on a {@link ConnectedStreams} and maps
-   * the output to a common type. The transformation calls a
-   * @param fun1 for each element of the first input and
-   * @param fun2 for each element of the second input. Each
-   * CoMapFunction call returns exactly one element.
-   *
-   * The CoMapFunction used to jointly transform the two input
-   * DataStreams
-   * @return The transformed { @link DataStream}
-   */
-  def map[R: TypeInformation: ClassTag](fun1: IN1 => R, fun2: IN2 => R): 
-  DataStream[R] = {
-    if (fun1 == null || fun2 == null) {
-      throw new NullPointerException("Map function must not be null.")
-    }
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    val comapper = new CoMapFunction[IN1, IN2, R] {
-      def map1(in1: IN1): R = cleanFun1(in1)
-      def map2(in2: IN2): R = cleanFun2(in2)
-    }
-
-    map(comapper)
-  }
-
-  /**
-   * Applies a CoMap transformation on a {@link ConnectedStreams} and maps
-   * the output to a common type. The transformation calls a
-   * {@link CoMapFunction#map1} for each element of the first input and
-   * {@link CoMapFunction#map2} for each element of the second input. Each
-   * CoMapFunction call returns exactly one element. The user can also extend
-   * {@link RichCoMapFunction} to gain access to other features provided by
-   * the {@link RichFuntion} interface.
-   *
-   * @param coMapper
-   * The CoMapFunction used to jointly transform the two input
-   * DataStreams
-   * @return The transformed { @link DataStream}
-   */
-  def map[R: TypeInformation: ClassTag](coMapper: CoMapFunction[IN1, IN2, R]): 
-  DataStream[R] = {
-    if (coMapper == null) {
-      throw new NullPointerException("Map function must not be null.")
-    }
-
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
-    javaStream.map(coMapper).returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
-   * maps the output to a common type. The transformation calls a
-   * {@link CoFlatMapFunction#flatMap1} for each element of the first input
-   * and {@link CoFlatMapFunction#flatMap2} for each element of the second
-   * input. Each CoFlatMapFunction call returns any number of elements
-   * including none. The user can also extend {@link RichFlatMapFunction} to
-   * gain access to other features provided by the {@link RichFuntion}
-   * interface.
-   *
-   * @param coFlatMapper
-   * The CoFlatMapFunction used to jointly transform the two input
-   * DataStreams
-   * @return The transformed { @link DataStream}
-   */
-  def flatMap[R: TypeInformation: ClassTag](coFlatMapper: CoFlatMapFunction[IN1, IN2, R]): 
-  DataStream[R] = {
-    if (coFlatMapper == null) {
-      throw new NullPointerException("FlatMap function must not be null.")
-    }
-    
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]    
-    javaStream.flatMap(coFlatMapper).returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
-   * maps the output to a common type. The transformation calls a
-   * @param fun1 for each element of the first input
-   * and @param fun2 for each element of the second
-   * input. Each CoFlatMapFunction call returns any number of elements
-   * including none.
-   *
-   * @return The transformed { @link DataStream}
-   */
-  def flatMap[R: TypeInformation: ClassTag](fun1: (IN1, Collector[R]) => Unit, 
-      fun2: (IN2, Collector[R]) => Unit): DataStream[R] = {
-    if (fun1 == null || fun2 == null) {
-      throw new NullPointerException("FlatMap functions must not be null.")
-    }
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    val flatMapper = new CoFlatMapFunction[IN1, IN2, R] {
-      def flatMap1(value: IN1, out: Collector[R]): Unit = cleanFun1(value, out)
-      def flatMap2(value: IN2, out: Collector[R]): Unit = cleanFun2(value, out)
-    }
-    flatMap(flatMapper)
-  }
-
-  /**
-   * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
-   * maps the output to a common type. The transformation calls a
-   * @param fun1 for each element of the first input
-   * and @param fun2 for each element of the second
-   * input. Each CoFlatMapFunction call returns any number of elements
-   * including none.
-   *
-   * @return The transformed { @link DataStream}
-   */
-  def flatMap[R: TypeInformation: ClassTag](fun1: IN1 => TraversableOnce[R],
-      fun2: IN2 => TraversableOnce[R]): DataStream[R] = {
-    if (fun1 == null || fun2 == null) {
-      throw new NullPointerException("FlatMap functions must not be null.")
-    }
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    val flatMapper = new CoFlatMapFunction[IN1, IN2, R] {
-      def flatMap1(value: IN1, out: Collector[R]) = { cleanFun1(value) foreach out.collect }
-      def flatMap2(value: IN2, out: Collector[R]) = { cleanFun2(value) foreach out.collect }
-    }
-    flatMap(flatMapper)
-  }
-
-  /**
-   * GroupBy operation for connected data stream. Groups the elements of
-   * input1 and input2 according to keyPosition1 and keyPosition2. Used for
-   * applying function on grouped data streams for example
-   * {@link ConnectedStreams#reduce}
-   *
-   * @param keyPosition1
-   * The field used to compute the hashcode of the elements in the
-   * first input stream.
-   * @param keyPosition2
-   * The field used to compute the hashcode of the elements in the
-   * second input stream.
-   * @return @return The transformed { @link ConnectedStreams}
-   */
-  def keyBy(keyPosition1: Int, keyPosition2: Int): ConnectedStreams[IN1, IN2] = {
-    javaStream.keyBy(keyPosition1, keyPosition2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream. Groups the elements of
-   * input1 and input2 according to keyPositions1 and keyPositions2. Used for
-   * applying function on grouped data streams for example
-   * {@link ConnectedStreams#reduce}
-   *
-   * @param keyPositions1
-   * The fields used to group the first input stream.
-   * @param keyPositions2
-   * The fields used to group the second input stream.
-   * @return @return The transformed { @link ConnectedStreams}
-   */
-  def keyBy(keyPositions1: Array[Int], keyPositions2: Array[Int]):
-  ConnectedStreams[IN1, IN2] = {
-    javaStream.keyBy(keyPositions1, keyPositions2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream using key expressions. Groups
-   * the elements of input1 and input2 according to field1 and field2. A field
-   * expression is either the name of a public field or a getter method with
-   * parentheses of the {@link DataStream}S underlying type. A dot can be used
-   * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-   *
-   * @param field1
-   * The grouping expression for the first input
-   * @param field2
-   * The grouping expression for the second input
-   * @return The grouped { @link ConnectedStreams}
-   */
-  def keyBy(field1: String, field2: String): ConnectedStreams[IN1, IN2] = {
-    javaStream.keyBy(field1, field2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream using key expressions. Groups
-   * the elements of input1 and input2 according to fields1 and fields2. A
-   * field expression is either the name of a public field or a getter method
-   * with parentheses of the {@link DataStream}S underlying type. A dot can be
-   * used to drill down into objects, as in {@code "field1.getInnerField2()" }
-   * .
-   *
-   * @param fields1
-   * The grouping expressions for the first input
-   * @param fields2
-   * The grouping expressions for the second input
-   * @return The grouped { @link ConnectedStreams}
-   */
-  def keyBy(fields1: Array[String], fields2: Array[String]):
-  ConnectedStreams[IN1, IN2] = {
-    javaStream.keyBy(fields1, fields2)
-  }
-
-  /**
-   * GroupBy operation for connected data stream. Groups the elements of
-   * input1 and input2 using fun1 and fun2. Used for applying
-   * function on grouped data streams for example
-   * {@link ConnectedStreams#reduce}
-   *
-   * @param fun1
-   * The function used for grouping the first input
-   * @param fun2
-   * The function used for grouping the second input
-   * @return The grouped { @link ConnectedStreams}
-   */
-  def keyBy[K1: TypeInformation, K2: TypeInformation](fun1: IN1 => K1, fun2: IN2 => K2):
-  ConnectedStreams[IN1, IN2] = {
-
-    val keyType1 = implicitly[TypeInformation[K1]]
-    val keyType2 = implicitly[TypeInformation[K2]]
-    
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-    
-    val keyExtractor1 = new KeySelectorWithType[IN1, K1](cleanFun1, keyType1)
-    val keyExtractor2 = new KeySelectorWithType[IN2, K2](cleanFun2, keyType2)
-    
-    javaStream.keyBy(keyExtractor1, keyExtractor2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream. Partitions the elements of
-   * input1 and input2 according to keyPosition1 and keyPosition2.
-   *
-   * @param keyPosition1
-   * The field used to compute the hashcode of the elements in the
-   * first input stream.
-   * @param keyPosition2
-   * The field used to compute the hashcode of the elements in the
-   * second input stream.
-   * @return The transformed { @link ConnectedStreams}
-   */
-  def partitionByHash(keyPosition1: Int, keyPosition2: Int): ConnectedStreams[IN1, IN2] = {
-    javaStream.partitionByHash(keyPosition1, keyPosition2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream. Partitions the elements of
-   * input1 and input2 according to keyPositions1 and keyPositions2.
-   *
-   * @param keyPositions1
-   * The fields used to partition the first input stream.
-   * @param keyPositions2
-   * The fields used to partition the second input stream.
-   * @return The transformed { @link ConnectedStreams}
-   */
-  def partitionByHash(keyPositions1: Array[Int], keyPositions2: Array[Int]):
-  ConnectedStreams[IN1, IN2] = {
-    javaStream.partitionByHash(keyPositions1, keyPositions2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream using key expressions. Partitions
-   * the elements of input1 and input2 according to field1 and field2. A field
-   * expression is either the name of a public field or a getter method with
-   * parentheses of the {@link DataStream}S underlying type. A dot can be used
-   * to drill down into objects, as in {@code "field1.getInnerField2()" }.
-   *
-   * @param field1
-   * The partitioning expression for the first input
-   * @param field2
-   * The partitioning expression for the second input
-   * @return The grouped { @link ConnectedStreams}
-   */
-  def partitionByHash(field1: String, field2: String): ConnectedStreams[IN1, IN2] = {
-    javaStream.partitionByHash(field1, field2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream using key expressions. Partitions
-   * the elements of input1 and input2 according to fields1 and fields2.
-   *
-   * @param fields1
-   * The partitioning expressions for the first input
-   * @param fields2
-   * The partitioning expressions for the second input
-   * @return The partitioned { @link ConnectedStreams}
-   */
-  def partitionByHash(fields1: Array[String], fields2: Array[String]):
-  ConnectedStreams[IN1, IN2] = {
-    javaStream.partitionByHash(fields1, fields2)
-  }
-
-  /**
-   * PartitionBy operation for connected data stream. Partitions the elements of
-   * input1 and input2 using fun1 and fun2.
-   *
-   * @param fun1
-   * The function used for partitioning the first input
-   * @param fun2
-   * The function used for partitioning the second input
-   * @return The partitioned { @link ConnectedStreams}
-   */
-  def partitionByHash[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
-  ConnectedStreams[IN1, IN2] = {
-
-    val cleanFun1 = clean(fun1)
-    val cleanFun2 = clean(fun2)
-
-    val keyExtractor1 = new KeySelector[IN1, K] {
-      def getKey(in: IN1) = cleanFun1(in)
-    }
-    val keyExtractor2 = new KeySelector[IN2, L] {
-      def getKey(in: IN2) = cleanFun2(in)
-    }
-
-    javaStream.partitionByHash(keyExtractor1, keyExtractor2)
-  }
-
-  /**
-   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig}
-   */
-  private[flink] def clean[F <: AnyRef](f: F): F = {
-    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
-  }
-
-}
-
-class KeySelectorWithType[IN, K](
-        private[this] val fun: IN => K,
-        private[this] val info: TypeInformation[K])
-  extends KeySelector[IN, K] with ResultTypeQueryable[K] {
-  
-  override def getKey(value: IN): K = fun(value)
-
-  override def getProducedType: TypeInformation[K] = info
-}


[46/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
deleted file mode 100644
index d511796..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ /dev/null
@@ -1,382 +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.connectors.kafka;
-
-import kafka.admin.AdminUtils;
-import kafka.common.KafkaException;
-import kafka.consumer.ConsumerConfig;
-import kafka.network.SocketServer;
-import kafka.server.KafkaConfig;
-import kafka.server.KafkaServer;
-
-import org.I0Itec.zkclient.ZkClient;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.curator.test.TestingServer;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.StreamingMode;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.kafka.internals.ZooKeeperStringSerializer;
-import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
-import org.apache.flink.util.NetUtils;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.kafka.common.PartitionInfo;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.BindException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.NetUtils.hostAndPortToUrlString;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * The base for the Kafka tests. It brings up:
- * <ul>
- *     <li>A ZooKeeper mini cluster</li>
- *     <li>Three Kafka Brokers (mini clusters)</li>
- *     <li>A Flink mini cluster</li>
- * </ul>
- * 
- * <p>Code in this test is based on the following GitHub repository:
- * <a href="https://github.com/sakserv/hadoop-mini-clusters">
- *   https://github.com/sakserv/hadoop-mini-clusters</a> (ASL licensed),
- * as per commit <i>bc6b2b2d5f6424d5f377aa6c0871e82a956462ef</i></p>
- */
-@SuppressWarnings("serial")
-public abstract class KafkaTestBase extends TestLogger {
-
-	protected static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class);
-	
-	protected static final int NUMBER_OF_KAFKA_SERVERS = 3;
-
-	protected static String zookeeperConnectionString;
-
-	protected static File tmpZkDir;
-
-	protected static File tmpKafkaParent;
-
-	protected static TestingServer zookeeper;
-	protected static List<KafkaServer> brokers;
-	protected static String brokerConnectionStrings = "";
-
-	protected static ConsumerConfig standardCC;
-	protected static Properties standardProps;
-	
-	protected static ForkableFlinkMiniCluster flink;
-
-	protected static int flinkPort;
-
-	protected static FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
-
-	protected static List<File> tmpKafkaDirs;
-
-	protected static String kafkaHost = "localhost";
-
-	// ------------------------------------------------------------------------
-	//  Setup and teardown of the mini clusters
-	// ------------------------------------------------------------------------
-	
-	@BeforeClass
-	public static void prepare() throws IOException {
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    Starting KafkaITCase ");
-		LOG.info("-------------------------------------------------------------------------");
-		
-		LOG.info("Starting KafkaITCase.prepare()");
-		
-		File tempDir = new File(System.getProperty("java.io.tmpdir"));
-		
-		tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString()));
-		assertTrue("cannot create zookeeper temp dir", tmpZkDir.mkdirs());
-
-		tmpKafkaParent = new File(tempDir, "kafkaITcase-kafka-dir*" + (UUID.randomUUID().toString()));
-		assertTrue("cannot create kafka temp dir", tmpKafkaParent.mkdirs());
-
-		tmpKafkaDirs = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS);
-		for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
-			File tmpDir = new File(tmpKafkaParent, "server-" + i);
-			assertTrue("cannot create kafka temp dir", tmpDir.mkdir());
-			tmpKafkaDirs.add(tmpDir);
-		}
-		
-		zookeeper = null;
-		brokers = null;
-
-		try {
-			LOG.info("Starting Zookeeper");
-			zookeeper = new TestingServer(-1, tmpZkDir);
-			zookeeperConnectionString = zookeeper.getConnectString();
-
-			LOG.info("Starting KafkaServer");
-			brokers = new ArrayList<>(NUMBER_OF_KAFKA_SERVERS);
-			
-			for (int i = 0; i < NUMBER_OF_KAFKA_SERVERS; i++) {
-				brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i), kafkaHost, zookeeperConnectionString));
-				SocketServer socketServer = brokers.get(i).socketServer();
-				
-				String host = socketServer.host() == null ? "localhost" : socketServer.host();
-				brokerConnectionStrings += hostAndPortToUrlString(host, socketServer.port()) + ",";
-			}
-
-			LOG.info("ZK and KafkaServer started.");
-		}
-		catch (Throwable t) {
-			t.printStackTrace();
-			fail("Test setup failed: " + t.getMessage());
-		}
-
-		standardProps = new Properties();
-
-		standardProps.setProperty("zookeeper.connect", zookeeperConnectionString);
-		standardProps.setProperty("bootstrap.servers", brokerConnectionStrings);
-		standardProps.setProperty("group.id", "flink-tests");
-		standardProps.setProperty("auto.commit.enable", "false");
-		standardProps.setProperty("zookeeper.session.timeout.ms", "12000"); // 6 seconds is default. Seems to be too small for travis.
-		standardProps.setProperty("zookeeper.connection.timeout.ms", "20000");
-		standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning.
-		standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!)
-		
-		Properties consumerConfigProps = new Properties();
-		consumerConfigProps.putAll(standardProps);
-		consumerConfigProps.setProperty("auto.offset.reset", "smallest");
-		standardCC = new ConsumerConfig(consumerConfigProps);
-		
-		// start also a re-usable Flink mini cluster
-		
-		Configuration flinkConfig = new Configuration();
-		flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
-		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
-		flinkConfig.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 s");
-
-		flink = new ForkableFlinkMiniCluster(flinkConfig, false, StreamingMode.STREAMING);
-		flink.start();
-
-		flinkPort = flink.getLeaderRPCPort();
-	}
-
-	@AfterClass
-	public static void shutDownServices() {
-
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    Shut down KafkaITCase ");
-		LOG.info("-------------------------------------------------------------------------");
-
-		flinkPort = -1;
-		if (flink != null) {
-			flink.shutdown();
-		}
-		
-		for (KafkaServer broker : brokers) {
-			if (broker != null) {
-				broker.shutdown();
-			}
-		}
-		brokers.clear();
-		
-		if (zookeeper != null) {
-			try {
-				zookeeper.stop();
-			}
-			catch (Exception e) {
-				LOG.warn("ZK.stop() failed", e);
-			}
-			zookeeper = null;
-		}
-		
-		// clean up the temp spaces
-		
-		if (tmpKafkaParent != null && tmpKafkaParent.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpKafkaParent);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-		if (tmpZkDir != null && tmpZkDir.exists()) {
-			try {
-				FileUtils.deleteDirectory(tmpZkDir);
-			}
-			catch (Exception e) {
-				// ignore
-			}
-		}
-
-		LOG.info("-------------------------------------------------------------------------");
-		LOG.info("    KafkaITCase finished"); 
-		LOG.info("-------------------------------------------------------------------------");
-	}
-
-	/**
-	 * Copied from com.github.sakserv.minicluster.KafkaLocalBrokerIntegrationTest (ASL licensed)
-	 */
-	protected static KafkaServer getKafkaServer(int brokerId, File tmpFolder,
-												String kafkaHost,
-												String zookeeperConnectionString) throws Exception {
-		Properties kafkaProperties = new Properties();
-
-		// properties have to be Strings
-		kafkaProperties.put("advertised.host.name", kafkaHost);
-		kafkaProperties.put("broker.id", Integer.toString(brokerId));
-		kafkaProperties.put("log.dir", tmpFolder.toString());
-		kafkaProperties.put("zookeeper.connect", zookeeperConnectionString);
-		kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024));
-		kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024));
-		
-		// for CI stability, increase zookeeper session timeout
-		kafkaProperties.put("zookeeper.session.timeout.ms", "20000");
-
-		final int numTries = 5;
-		
-		for (int i = 1; i <= numTries; i++) { 
-			int kafkaPort = NetUtils.getAvailablePort();
-			kafkaProperties.put("port", Integer.toString(kafkaPort));
-			KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties);
-
-			try {
-				KafkaServer server = new KafkaServer(kafkaConfig, new KafkaLocalSystemTime());
-				server.startup();
-				return server;
-			}
-			catch (KafkaException e) {
-				if (e.getCause() instanceof BindException) {
-					// port conflict, retry...
-					LOG.info("Port conflict when starting Kafka Broker. Retrying...");
-				}
-				else {
-					throw e;
-				}
-			}
-		}
-		
-		throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts.");
-	}
-
-	// ------------------------------------------------------------------------
-	//  Execution utilities
-	// ------------------------------------------------------------------------
-	
-	protected ZkClient createZookeeperClient() {
-		return new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
-				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
-	}
-	
-	protected static void tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
-		try {
-			see.execute(name);
-		}
-		catch (ProgramInvocationException | JobExecutionException root) {
-			Throwable cause = root.getCause();
-			
-			// search for nested SuccessExceptions
-			int depth = 0;
-			while (!(cause instanceof SuccessException)) {
-				if (cause == null || depth++ == 20) {
-					root.printStackTrace();
-					fail("Test failed: " + root.getMessage());
-				}
-				else {
-					cause = cause.getCause();
-				}
-			}
-		}
-	}
-
-	protected static void tryExecutePropagateExceptions(StreamExecutionEnvironment see, String name) throws Exception {
-		try {
-			see.execute(name);
-		}
-		catch (ProgramInvocationException | JobExecutionException root) {
-			Throwable cause = root.getCause();
-
-			// search for nested SuccessExceptions
-			int depth = 0;
-			while (!(cause instanceof SuccessException)) {
-				if (cause == null || depth++ == 20) {
-					throw root;
-				}
-				else {
-					cause = cause.getCause();
-				}
-			}
-		}
-	}
-	
-	
-
-	protected static void createTestTopic(String topic, int numberOfPartitions, int replicationFactor) {
-		
-		// create topic with one client
-		Properties topicConfig = new Properties();
-		LOG.info("Creating topic {}", topic);
-
-		ZkClient creator = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
-				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
-		
-		AdminUtils.createTopic(creator, topic, numberOfPartitions, replicationFactor, topicConfig);
-		creator.close();
-		
-		// validate that the topic has been created
-		final long deadline = System.currentTimeMillis() + 30000;
-		do {
-			try {
-				Thread.sleep(100);
-			}
-			catch (InterruptedException e) {
-				// restore interrupted state
-			}
-			List<PartitionInfo> partitions = FlinkKafkaConsumer.getPartitionsForTopic(topic, standardProps);
-			if (partitions != null && partitions.size() > 0) {
-				return;
-			}
-		}
-		while (System.currentTimeMillis() < deadline);
-		fail ("Test topic could not be created");
-	}
-	
-	protected static void deleteTestTopic(String topic) {
-		LOG.info("Deleting topic {}", topic);
-
-		ZkClient zk = new ZkClient(standardCC.zkConnect(), standardCC.zkSessionTimeoutMs(),
-				standardCC.zkConnectionTimeoutMs(), new ZooKeeperStringSerializer());
-
-		AdminUtils.deleteTopic(zk, topic);
-		
-		zk.close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java
deleted file mode 100644
index 75fdd46..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/TestFixedPartitioner.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
-
-public class TestFixedPartitioner {
-
-
-	/**
-	 * <pre>
-	 *   		Flink Sinks:		Kafka Partitions
-	 * 			1	---------------->	1
-	 * 			2   --------------/
-	 * 			3   -------------/
-	 * 			4	------------/
-	 * </pre>
-	 */
-	@Test
-	public void testMoreFlinkThanBrokers() {
-		FixedPartitioner part = new FixedPartitioner();
-
-		int[] partitions = new int[]{0};
-
-		part.open(0, 4, partitions);
-		Assert.assertEquals(0, part.partition("abc1", partitions.length));
-
-		part.open(1, 4, partitions);
-		Assert.assertEquals(0, part.partition("abc2", partitions.length));
-
-		part.open(2, 4, partitions);
-		Assert.assertEquals(0, part.partition("abc3", partitions.length));
-		Assert.assertEquals(0, part.partition("abc3", partitions.length)); // check if it is changing ;)
-
-		part.open(3, 4, partitions);
-		Assert.assertEquals(0, part.partition("abc4", partitions.length));
-	}
-
-	/**
-	 *
-	 * <pre>
-	 * 		Flink Sinks:		Kafka Partitions
-	 * 			1	---------------->	1
-	 * 			2	---------------->	2
-	 * 									3
-	 * 									4
-	 * 									5
-	 *
-	 * </pre>
-	 */
-	@Test
-	public void testFewerPartitions() {
-		FixedPartitioner part = new FixedPartitioner();
-
-		int[] partitions = new int[]{0, 1, 2, 3, 4};
-		part.open(0, 2, partitions);
-		Assert.assertEquals(0, part.partition("abc1", partitions.length));
-		Assert.assertEquals(0, part.partition("abc1", partitions.length));
-
-		part.open(1, 2, partitions);
-		Assert.assertEquals(1, part.partition("abc1", partitions.length));
-		Assert.assertEquals(1, part.partition("abc1", partitions.length));
-	}
-
-	/*
-	 * 		Flink Sinks:		Kafka Partitions
-	 * 			1	------------>--->	1
-	 * 			2	-----------/----> 	2
-	 * 			3	----------/
-	 */
-	@Test
-	public void testMixedCase() {
-		FixedPartitioner part = new FixedPartitioner();
-		int[] partitions = new int[]{0,1};
-
-		part.open(0, 3, partitions);
-		Assert.assertEquals(0, part.partition("abc1", partitions.length));
-
-		part.open(1, 3, partitions);
-		Assert.assertEquals(1, part.partition("abc1", partitions.length));
-
-		part.open(2, 3, partitions);
-		Assert.assertEquals(0, part.partition("abc1", partitions.length));
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
deleted file mode 100644
index 27ad2e8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandlerTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import kafka.admin.AdminUtils;
-
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.flink.streaming.connectors.kafka.KafkaTestBase;
-
-import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
-import org.junit.Test;
-
-import java.util.Properties;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public class ZookeeperOffsetHandlerTest extends KafkaTestBase {
-	
-	@Test
-	public void runOffsetManipulationinZooKeeperTest() {
-		try {
-			final String topicName = "ZookeeperOffsetHandlerTest-Topic";
-			final String groupId = "ZookeeperOffsetHandlerTest-Group";
-			
-			final long offset = (long) (Math.random() * Long.MAX_VALUE);
-
-			ZkClient zkClient = createZookeeperClient();
-			AdminUtils.createTopic(zkClient, topicName, 3, 2, new Properties());
-				
-			ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, groupId, topicName, 0, offset);
-	
-			long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, groupId, topicName, 0);
-
-			zkClient.close();
-			
-			assertEquals(offset, fetchedOffset);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
deleted file mode 100644
index 32377ae..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
+++ /dev/null
@@ -1,214 +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.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-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.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-
-import java.util.Random;
-
-@SuppressWarnings("serial")
-public class DataGenerators {
-	
-	public static void generateLongStringTupleSequence(StreamExecutionEnvironment env,
-														String brokerConnection, String topic,
-														int numPartitions,
-														final int from, final int to) throws Exception {
-
-		TypeInformation<Tuple2<Integer, Integer>> resultType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-
-		env.setParallelism(numPartitions);
-		env.getConfig().disableSysoutLogging();
-		env.setNumberOfExecutionRetries(0);
-		
-		DataStream<Tuple2<Integer, Integer>> stream =env.addSource(
-				new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
-
-					private volatile boolean running = true;
-
-					@Override
-					public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-						int cnt = from;
-						int partition = getRuntimeContext().getIndexOfThisSubtask();
-
-						while (running && cnt <= to) {
-							ctx.collect(new Tuple2<Integer, Integer>(partition, cnt));
-							cnt++;
-						}
-					}
-
-					@Override
-					public void cancel() {
-						running = false;
-					}
-				});
-
-		stream.addSink(new FlinkKafkaProducer<>(topic,
-				new TypeInformationSerializationSchema<>(resultType, env.getConfig()),
-				FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection),
-				new Tuple2Partitioner(numPartitions)
-		));
-
-		env.execute("Data generator (Int, Int) stream to topic " + topic);
-	}
-
-	// ------------------------------------------------------------------------
-	
-	public static void generateRandomizedIntegerSequence(StreamExecutionEnvironment env,
-															String brokerConnection, String topic,
-															final int numPartitions,
-															final int numElements,
-															final boolean randomizeOrder) throws Exception {
-		env.setParallelism(numPartitions);
-		env.getConfig().disableSysoutLogging();
-		env.setNumberOfExecutionRetries(0);
-
-		DataStream<Integer> stream = env.addSource(
-				new RichParallelSourceFunction<Integer>() {
-
-					private volatile boolean running = true;
-
-					@Override
-					public void run(SourceContext<Integer> ctx) {
-						// create a sequence
-						int[] elements = new int[numElements];
-						for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask();
-								i < numElements;
-								i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) {
-							
-							elements[i] = val;
-						}
-
-						// scramble the sequence
-						if (randomizeOrder) {
-							Random rnd = new Random();
-							for (int i = 0; i < elements.length; i++) {
-								int otherPos = rnd.nextInt(elements.length);
-								
-								int tmp = elements[i];
-								elements[i] = elements[otherPos];
-								elements[otherPos] = tmp;
-							}
-						}
-
-						// emit the sequence
-						int pos = 0;
-						while (running && pos < elements.length) {
-							ctx.collect(elements[pos++]);
-						}
-					}
-
-					@Override
-					public void cancel() {
-						running = false;
-					}
-				});
-
-		stream
-				.rebalance()
-				.addSink(new FlinkKafkaProducer<>(topic,
-						new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig()),
-						FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection),
-						new KafkaPartitioner() {
-							@Override
-							public int partition(Object key, int numPartitions) {
-								return ((Integer) key) % numPartitions;
-							}
-						}));
-
-		env.execute("Scrambles int sequence generator");
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	public static class InfiniteStringsGenerator extends Thread {
-
-		private final String kafkaConnectionString;
-		
-		private final String topic;
-		
-		private volatile Throwable error;
-		
-		private volatile boolean running = true;
-
-		
-		public InfiniteStringsGenerator(String kafkaConnectionString, String topic) {
-			this.kafkaConnectionString = kafkaConnectionString;
-			this.topic = topic;
-		}
-
-		@Override
-		public void run() {
-			// we manually feed data into the Kafka sink
-			FlinkKafkaProducer<String> producer = null;
-			try {
-				producer = new FlinkKafkaProducer<>(kafkaConnectionString, topic, new JavaDefaultStringSchema());
-				producer.setRuntimeContext(new MockRuntimeContext(1,0));
-				producer.open(new Configuration());
-				
-				final StringBuilder bld = new StringBuilder();
-				final Random rnd = new Random();
-				
-				while (running) {
-					bld.setLength(0);
-					
-					int len = rnd.nextInt(100) + 1;
-					for (int i = 0; i < len; i++) {
-						bld.append((char) (rnd.nextInt(20) + 'a') );
-					}
-					
-					String next = bld.toString();
-					producer.invoke(next);
-				}
-			}
-			catch (Throwable t) {
-				this.error = t;
-			}
-			finally {
-				if (producer != null) {
-					try {
-						producer.close();
-					}
-					catch (Throwable t) {
-						// ignore
-					}
-				}
-			}
-		}
-		
-		public void shutdown() {
-			this.running = false;
-			this.interrupt();
-		}
-		
-		public Throwable getError() {
-			return this.error;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java
deleted file mode 100644
index 987e6c5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-
-/**
- * Sink function that discards data.
- * @param <T> The type of the function.
- */
-public class DiscardingSink<T> implements SinkFunction<T> {
-
-	private static final long serialVersionUID = 2777597566520109843L;
-
-	@Override
-	public void invoke(T value) {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
deleted file mode 100644
index 5a8ffaa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
+++ /dev/null
@@ -1,115 +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.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class FailingIdentityMapper<T> extends RichMapFunction<T,T> implements
-		Checkpointed<Integer>, CheckpointNotifier, Runnable {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(FailingIdentityMapper.class);
-	
-	private static final long serialVersionUID = 6334389850158707313L;
-	
-	public static volatile boolean failedBefore;
-	public static volatile boolean hasBeenCheckpointedBeforeFailure;
-
-	private final int failCount;
-	private int numElementsTotal;
-	private int numElementsThisTime;
-	
-	private boolean failer;
-	private boolean hasBeenCheckpointed;
-	
-	private Thread printer;
-	private volatile boolean printerRunning = true;
-
-	public FailingIdentityMapper(int failCount) {
-		this.failCount = failCount;
-	}
-
-	@Override
-	public void open(Configuration parameters) {
-		failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
-		printer = new Thread(this, "FailingIdentityMapper Status Printer");
-		printer.start();
-	}
-
-	@Override
-	public T map(T value) throws Exception {
-		numElementsTotal++;
-		numElementsThisTime++;
-		
-		if (!failedBefore) {
-			Thread.sleep(10);
-			
-			if (failer && numElementsTotal >= failCount) {
-				hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed;
-				failedBefore = true;
-				throw new Exception("Artificial Test Failure");
-			}
-		}
-		return value;
-	}
-
-	@Override
-	public void close() throws Exception {
-		printerRunning = false;
-		if (printer != null) {
-			printer.interrupt();
-			printer = null;
-		}
-	}
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) {
-		this.hasBeenCheckpointed = true;
-	}
-
-	@Override
-	public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-		return numElementsTotal;
-	}
-
-	@Override
-	public void restoreState(Integer state) {
-		numElementsTotal = state;
-	}
-
-	@Override
-	public void run() {
-		while (printerRunning) {
-			try {
-				Thread.sleep(5000);
-			}
-			catch (InterruptedException e) {
-				// ignore
-			}
-			LOG.info("============================> Failing mapper  {}: count={}, totalCount={}",
-					getRuntimeContext().getIndexOfThisSubtask(),
-					numElementsThisTime, numElementsTotal);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
deleted file mode 100644
index e94adb5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
+++ /dev/null
@@ -1,76 +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.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.client.JobStatusMessage;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public class JobManagerCommunicationUtils {
-	
-	private static final FiniteDuration askTimeout = new FiniteDuration(30, TimeUnit.SECONDS);
-	
-	
-	public static void cancelCurrentJob(ActorGateway jobManager) throws Exception {
-		
-		// find the jobID
-		Future<Object> listResponse = jobManager.ask(
-				JobManagerMessages.getRequestRunningJobsStatus(),
-				askTimeout);
-
-		List<JobStatusMessage> jobs;
-		try {
-			Object result = Await.result(listResponse, askTimeout);
-			jobs = ((JobManagerMessages.RunningJobsStatus) result).getStatusMessages();
-		}
-		catch (Exception e) {
-			throw new Exception("Could not cancel job - failed to retrieve running jobs from the JobManager.", e);
-		}
-		
-		if (jobs.isEmpty()) {
-			throw new Exception("Could not cancel job - no running jobs");
-		}
-		if (jobs.size() != 1) {
-			throw new Exception("Could not cancel job - more than one running job.");
-		}
-		
-		JobStatusMessage status = jobs.get(0);
-		if (status.getJobState().isTerminalState()) {
-			throw new Exception("Could not cancel job - job is not running any more");
-		}
-		
-		JobID jobId = status.getJobId();
-		
-		Future<Object> response = jobManager.ask(new JobManagerMessages.CancelJob(jobId), askTimeout);
-		try {
-			Await.result(response, askTimeout);
-		}
-		catch (Exception e) {
-			throw new Exception("Sending the 'cancel' message failed.", e);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
deleted file mode 100644
index b9fc3de..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.accumulators.DoubleCounter;
-import org.apache.flink.api.common.accumulators.Histogram;
-import org.apache.flink.api.common.accumulators.IntCounter;
-import org.apache.flink.api.common.accumulators.LongCounter;
-import org.apache.flink.api.common.cache.DistributedCache;
-import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-public class MockRuntimeContext implements RuntimeContext {
-
-	private final int numberOfParallelSubtasks;
-	private final int indexOfThisSubtask;
-
-	public MockRuntimeContext(int numberOfParallelSubtasks, int indexOfThisSubtask) {
-		this.numberOfParallelSubtasks = numberOfParallelSubtasks;
-		this.indexOfThisSubtask = indexOfThisSubtask;
-	}
-
-
-	@Override
-	public String getTaskName() {
-		return null;
-	}
-
-	@Override
-	public int getNumberOfParallelSubtasks() {
-		return numberOfParallelSubtasks;
-	}
-
-	@Override
-	public int getIndexOfThisSubtask() {
-		return indexOfThisSubtask;
-	}
-
-	@Override
-	public ExecutionConfig getExecutionConfig() {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public ClassLoader getUserCodeClassLoader() {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public Map<String, Accumulator<?, ?>> getAllAccumulators() {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public IntCounter getIntCounter(String name) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public LongCounter getLongCounter(String name) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public DoubleCounter getDoubleCounter(String name) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public Histogram getHistogram(String name) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public <RT> List<RT> getBroadcastVariable(String name) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public DistributedCache getDistributedCache() {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
-		throw new UnsupportedOperationException();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
deleted file mode 100644
index e105e01..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.functions.MapFunction;
-
-import java.util.HashSet;
-import java.util.Set;
-
-
-public class PartitionValidatingMapper implements MapFunction<Integer, Integer> {
-
-	private static final long serialVersionUID = 1088381231244959088L;
-	
-	/* the partitions from which this function received data */
-	private final Set<Integer> myPartitions = new HashSet<>();
-	
-	private final int numPartitions;
-	private final int maxPartitions;
-
-	public PartitionValidatingMapper(int numPartitions, int maxPartitions) {
-		this.numPartitions = numPartitions;
-		this.maxPartitions = maxPartitions;
-	}
-
-	@Override
-	public Integer map(Integer value) throws Exception {
-		// validate that the partitioning is identical
-		int partition = value % numPartitions;
-		myPartitions.add(partition);
-		if (myPartitions.size() > maxPartitions) {
-			throw new Exception("Error: Elements from too many different partitions: " + myPartitions
-					+ ". Expect elements only from " + maxPartitions + " partitions");
-		}
-		return value;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java
deleted file mode 100644
index 12e3460..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-/**
- * Exception that is thrown to terminate a program and indicate success.
- */
-public class SuccessException extends Exception {
-	private static final long serialVersionUID = -7011865671593955887L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
deleted file mode 100644
index 1d61229..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.functions.MapFunction;
-
-/**
- * An identity map function that sleeps between elements, throttling the
- * processing speed.
- * 
- * @param <T> The type mapped.
- */
-public class ThrottledMapper<T> implements MapFunction<T,T> {
-
-	private static final long serialVersionUID = 467008933767159126L;
-
-	private final int sleep;
-
-	public ThrottledMapper(int sleep) {
-		this.sleep = sleep;
-	}
-
-	@Override
-	public T map(T value) throws Exception {
-		Thread.sleep(this.sleep);
-		return value;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
deleted file mode 100644
index b762e21..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-
-import java.io.Serializable;
-
-/**
- * Special partitioner that uses the first field of a 2-tuple as the partition,
- * and that expects a specific number of partitions.
- */
-public class Tuple2Partitioner extends KafkaPartitioner implements Serializable {
-	
-	private static final long serialVersionUID = 1L;
-
-	private final int expectedPartitions;
-
-	
-	public Tuple2Partitioner(int expectedPartitions) {
-		this.expectedPartitions = expectedPartitions;
-	}
-
-	@Override
-	public int partition(Object key, int numPartitions) {
-		if (numPartitions != expectedPartitions) {
-			throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions");
-		}
-		@SuppressWarnings("unchecked")
-		Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
-		
-		return element.f0;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
deleted file mode 100644
index f3cc4fa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.BitSet;
-
-public class ValidatingExactlyOnceSink implements SinkFunction<Integer>, Checkpointed<Tuple2<Integer, BitSet>> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ValidatingExactlyOnceSink.class);
-
-	private static final long serialVersionUID = 1748426382527469932L;
-	
-	private final int numElementsTotal;
-	
-	private BitSet duplicateChecker = new BitSet();  // this is checkpointed
-
-	private int numElements; // this is checkpointed
-
-	
-	public ValidatingExactlyOnceSink(int numElementsTotal) {
-		this.numElementsTotal = numElementsTotal;
-	}
-
-	
-	@Override
-	public void invoke(Integer value) throws Exception {
-		numElements++;
-		
-		if (duplicateChecker.get(value)) {
-			throw new Exception("Received a duplicate");
-		}
-		duplicateChecker.set(value);
-		if (numElements == numElementsTotal) {
-			// validate
-			if (duplicateChecker.cardinality() != numElementsTotal) {
-				throw new Exception("Duplicate checker has wrong cardinality");
-			}
-			else if (duplicateChecker.nextClearBit(0) != numElementsTotal) {
-				throw new Exception("Received sparse sequence");
-			}
-			else {
-				throw new SuccessException();
-			}
-		}
-	}
-
-	@Override
-	public Tuple2<Integer, BitSet> snapshotState(long checkpointId, long checkpointTimestamp) {
-		LOG.info("Snapshot of counter "+numElements+" at checkpoint "+checkpointId);
-		return new Tuple2<>(numElements, duplicateChecker);
-	}
-
-	@Override
-	public void restoreState(Tuple2<Integer, BitSet> state) {
-		LOG.info("restoring num elements to {}", state.f0);
-		this.numElements = state.f0;
-		this.duplicateChecker = state.f1;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
deleted file mode 100644
index 6bdfb48..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,29 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-log4j.rootLogger=INFO, testlogger
-
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
-
-# suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
-
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml
deleted file mode 100644
index 45b3b92..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +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.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.streaming" level="WARN"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/pom.xml
deleted file mode 100644
index 9168822..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/pom.xml
+++ /dev/null
@@ -1,94 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-nifi</artifactId>
-	<name>flink-connector-nifi</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<nifi.version>0.3.0</nifi.version>
-	</properties>
-
-	<dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-site-to-site-client</artifactId>
-            <version>${nifi.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-streaming-core</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-tests</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-test-utils</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<rerunFailingTestsCount>3</rerunFailingTestsCount>
-				</configuration>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-failsafe-plugin</artifactId>
-				<configuration>
-					<rerunFailingTestsCount>3</rerunFailingTestsCount>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java
deleted file mode 100644
index c8ceb57..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.nifi;
-
-import java.util.Map;
-
-/**
- * <p>
- * The NiFiDataPacket provides a packaging around a NiFi FlowFile. It wraps both
- * a FlowFile's content and its attributes so that they can be processed by Flink.
- * </p>
- */
-public interface NiFiDataPacket {
-
-	/**
-	 * @return the contents of a NiFi FlowFile
-	 */
-	byte[] getContent();
-
-	/**
-	 * @return a Map of attributes that are associated with the NiFi FlowFile
-	 */
-	Map<String, String> getAttributes();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java
deleted file mode 100644
index 9bb521b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.nifi;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.RuntimeContext;
-
-import java.io.Serializable;
-
-/**
- * A function that can create a NiFiDataPacket from an incoming instance of the given type.
- *
- * @param <T>
- */
-public interface NiFiDataPacketBuilder<T> extends Function, Serializable {
-
-	NiFiDataPacket createNiFiDataPacket(T t, RuntimeContext ctx);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java
deleted file mode 100644
index abc6b35..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.nifi;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.nifi.remote.Transaction;
-import org.apache.nifi.remote.TransferDirection;
-import org.apache.nifi.remote.client.SiteToSiteClient;
-import org.apache.nifi.remote.client.SiteToSiteClientConfig;
-
-/**
- * A sink that delivers data to Apache NiFi using the NiFi Site-to-Site client. The sink requires
- * a NiFiDataPacketBuilder which can create instances of NiFiDataPacket from the incoming data.
- */
-public class NiFiSink<T> extends RichSinkFunction<T> {
-
-	private SiteToSiteClient client;
-	private SiteToSiteClientConfig clientConfig;
-	private NiFiDataPacketBuilder<T> builder;
-
-	/**
-	 * Construct a new NiFiSink with the given client config and NiFiDataPacketBuilder.
-	 *
-	 * @param clientConfig the configuration for building a NiFi SiteToSiteClient
-	 * @param builder a builder to produce NiFiDataPackets from incoming data
-	 */
-	public NiFiSink(SiteToSiteClientConfig clientConfig, NiFiDataPacketBuilder<T> builder) {
-		this.clientConfig = clientConfig;
-		this.builder = builder;
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		this.client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build();
-	}
-
-	@Override
-	public void invoke(T value) throws Exception {
-		final NiFiDataPacket niFiDataPacket = builder.createNiFiDataPacket(value, getRuntimeContext());
-
-		final Transaction transaction = client.createTransaction(TransferDirection.SEND);
-		if (transaction == null) {
-			throw new IllegalStateException("Unable to create a NiFi Transaction to send data");
-		}
-
-		transaction.send(niFiDataPacket.getContent(), niFiDataPacket.getAttributes());
-		transaction.confirm();
-		transaction.complete();
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		client.close();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java
deleted file mode 100644
index a213bb4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java
+++ /dev/null
@@ -1,146 +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.connectors.nifi;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.nifi.remote.Transaction;
-import org.apache.nifi.remote.TransferDirection;
-import org.apache.nifi.remote.client.SiteToSiteClient;
-import org.apache.nifi.remote.client.SiteToSiteClientConfig;
-import org.apache.nifi.remote.protocol.DataPacket;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A source that pulls data from Apache NiFi using the NiFi Site-to-Site client. This source
- * produces NiFiDataPackets which encapsulate the content and attributes of a NiFi FlowFile.
- */
-public class NiFiSource extends RichParallelSourceFunction<NiFiDataPacket> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(NiFiSource.class);
-
-	private static final long DEFAULT_WAIT_TIME_MS = 1000;
-
-	private long waitTimeMs;
-	private SiteToSiteClient client;
-	private SiteToSiteClientConfig clientConfig;
-	private transient volatile boolean running;
-
-	/**
-	 * Constructs a new NiFiSource using the given client config and the default wait time of 1000 ms.
-	 *
-	 * @param clientConfig the configuration for building a NiFi SiteToSiteClient
-	 */
-	public NiFiSource(SiteToSiteClientConfig clientConfig) {
-		this(clientConfig, DEFAULT_WAIT_TIME_MS);
-	}
-
-	/**
-	 * Constructs a new NiFiSource using the given client config and wait time.
-	 *
-	 * @param clientConfig the configuration for building a NiFi SiteToSiteClient
-	 * @param waitTimeMs the amount of time to wait (in milliseconds) if no data is available to pull from NiFi
-	 */
-	public NiFiSource(SiteToSiteClientConfig clientConfig, long waitTimeMs) {
-		this.clientConfig = clientConfig;
-		this.waitTimeMs = waitTimeMs;
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build();
-		running = true;
-	}
-
-	@Override
-	public void run(SourceContext<NiFiDataPacket> ctx) throws Exception {
-		try {
-			while (running) {
-				final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
-				if (transaction == null) {
-					LOG.warn("A transaction could not be created, waiting and will try again...");
-					try {
-						Thread.sleep(waitTimeMs);
-					} catch (InterruptedException e) {
-
-					}
-					continue;
-				}
-
-				DataPacket dataPacket = transaction.receive();
-				if (dataPacket == null) {
-					transaction.confirm();
-					transaction.complete();
-
-					LOG.debug("No data available to pull, waiting and will try again...");
-					try {
-						Thread.sleep(waitTimeMs);
-					} catch (InterruptedException e) {
-
-					}
-					continue;
-				}
-
-				final List<NiFiDataPacket> niFiDataPackets = new ArrayList<>();
-				do {
-					// Read the data into a byte array and wrap it along with the attributes
-					// into a NiFiDataPacket.
-					final InputStream inStream = dataPacket.getData();
-					final byte[] data = new byte[(int) dataPacket.getSize()];
-					StreamUtils.fillBuffer(inStream, data);
-
-					final Map<String, String> attributes = dataPacket.getAttributes();
-
-					niFiDataPackets.add(new StandardNiFiDataPacket(data, attributes));
-					dataPacket = transaction.receive();
-				} while (dataPacket != null);
-
-				// Confirm transaction to verify the data
-				transaction.confirm();
-
-				for (NiFiDataPacket dp : niFiDataPackets) {
-					ctx.collect(dp);
-				}
-
-				transaction.complete();
-			}
-		} finally {
-			ctx.close();
-		}
-	}
-
-	@Override
-	public void cancel() {
-		running = false;
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		client.close();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java
deleted file mode 100644
index 5ad4bae..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.nifi;
-
-import java.io.Serializable;
-import java.util.Map;
-
-/**
- * An implementation of NiFiDataPacket.
- */
-public class StandardNiFiDataPacket implements NiFiDataPacket, Serializable {
-	private static final long serialVersionUID = 6364005260220243322L;
-
-	private final byte[] content;
-	private final Map<String, String> attributes;
-
-	public StandardNiFiDataPacket(final byte[] content, final Map<String, String> attributes) {
-		this.content = content;
-		this.attributes = attributes;
-	}
-
-	@Override
-	public byte[] getContent() {
-		return content;
-	}
-
-	@Override
-	public Map<String, String> getAttributes() {
-		return attributes;
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java
deleted file mode 100644
index 572f949..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.flink.streaming.connectors.nifi.examples;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.nifi.NiFiDataPacket;
-import org.apache.flink.streaming.connectors.nifi.NiFiDataPacketBuilder;
-import org.apache.flink.streaming.connectors.nifi.NiFiSink;
-import org.apache.flink.streaming.connectors.nifi.StandardNiFiDataPacket;
-import org.apache.nifi.remote.client.SiteToSiteClient;
-import org.apache.nifi.remote.client.SiteToSiteClientConfig;
-
-import java.util.HashMap;
-
-/**
- * An example topology that sends data to a NiFi input port named "Data from Flink".
- */
-public class NiFiSinkTopologyExample {
-
-	public static void main(String[] args) throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		SiteToSiteClientConfig clientConfig = new SiteToSiteClient.Builder()
-				.url("http://localhost:8080/nifi")
-				.portName("Data from Flink")
-				.buildConfig();
-
-		DataStreamSink<String> dataStream = env.fromElements("one", "two", "three", "four", "five", "q")
-				.addSink(new NiFiSink<>(clientConfig, new NiFiDataPacketBuilder<String>() {
-					@Override
-					public NiFiDataPacket createNiFiDataPacket(String s, RuntimeContext ctx) {
-						return new StandardNiFiDataPacket(s.getBytes(), new HashMap<String,String>());
-					}
-				}));
-
-		env.execute();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.java
deleted file mode 100644
index 79c9a1c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.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.connectors.nifi.examples;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.nifi.NiFiDataPacket;
-import org.apache.flink.streaming.connectors.nifi.NiFiSource;
-import org.apache.nifi.remote.client.SiteToSiteClient;
-import org.apache.nifi.remote.client.SiteToSiteClientConfig;
-
-import java.nio.charset.Charset;
-
-/**
- * An example topology that pulls data from a NiFi output port named "Data for Flink".
- */
-public class NiFiSourceTopologyExample {
-
-	public static void main(String[] args) throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		SiteToSiteClientConfig clientConfig = new SiteToSiteClient.Builder()
-				.url("http://localhost:8080/nifi")
-				.portName("Data for Flink")
-				.requestBatchCount(5)
-				.buildConfig();
-
-		SourceFunction<NiFiDataPacket> nifiSource = new NiFiSource(clientConfig);
-		DataStream<NiFiDataPacket> streamSource = env.addSource(nifiSource).setParallelism(2);
-
-		DataStream<String> dataStream = streamSource.map(new MapFunction<NiFiDataPacket, String>() {
-			@Override
-			public String map(NiFiDataPacket value) throws Exception {
-				return new String(value.getContent(), Charset.defaultCharset());
-			}
-		});
-
-		dataStream.print();
-		env.execute();
-	}
-
-}


[32/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/StreamingOperatorMetrics.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/StreamingOperatorMetrics.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/StreamingOperatorMetrics.java
deleted file mode 100644
index 8429889..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/StreamingOperatorMetrics.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators;
-
-public class StreamingOperatorMetrics {
-	
-	
-	public void incrementLateElementDiscarded() {
-		
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java
deleted file mode 100644
index 50d1cb6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/Triggerable.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators;
-
-/**
- * This interface must be implemented by objects that are triggered by the timer service available
- * to stream operators in {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment}.
- */
-public interface Triggerable {
-
-	/**
-	 * This method is invoked with the timestamp for which the trigger was scheduled.
-	 * <p>
-	 * If the triggering is delayed for whatever reason (trigger timer was blocked, JVM stalled due
-	 * to a garbage collection), the timestamp supplied to this function will still be the original
-	 * timestamp for which the trigger was scheduled.
-	 * 
-	 * @param timestamp The timestamp for which the trigger event was scheduled.
-	 */
-	void trigger(long timestamp) throws Exception ;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/package-info.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/package-info.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/package-info.java
deleted file mode 100644
index 5fe6873..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * This package contains the operators that perform the stream transformations.
- * One or more operators are bundled into a "chain" and executed in a stream task. 
- */
-package org.apache.flink.streaming.runtime.operators;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
deleted file mode 100644
index 3165f88..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
+++ /dev/null
@@ -1,335 +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.math3.util.ArithmeticUtils;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.util.MathUtils;
-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.runtime.state.StateBackend;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-
-import static java.util.Objects.requireNonNull;
-
-public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, STATE, F extends Function> 
-		extends AbstractUdfStreamOperator<OUT, F> 
-		implements OneInputStreamOperator<IN, OUT>, Triggerable {
-	
-	private static final long serialVersionUID = 3245500864882459867L;
-	
-	private static final long MIN_SLIDE_TIME = 50;
-	
-	// ----- fields for operator parametrization -----
-	
-	private final Function function;
-	private final KeySelector<IN, KEY> keySelector;
-	
-	private final TypeSerializer<KEY> keySerializer;
-	private final TypeSerializer<STATE> stateTypeSerializer;
-	
-	private final long windowSize;
-	private final long windowSlide;
-	private final long paneSize;
-	private final int numPanesPerWindow;
-	
-	// ----- fields for operator functionality -----
-	
-	private transient AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes;
-	
-	private transient TimestampedCollector<OUT> out;
-	
-	private transient RestoredState<IN, KEY, STATE, OUT> restoredState;
-	
-	private transient long nextEvaluationTime;
-	private transient long nextSlideTime;
-	
-	protected AbstractAlignedProcessingTimeWindowOperator(
-			F function,
-			KeySelector<IN, KEY> keySelector,
-			TypeSerializer<KEY> keySerializer,
-			TypeSerializer<STATE> stateTypeSerializer,
-			long windowLength,
-			long windowSlide)
-	{
-		super(function);
-		
-		if (windowLength < MIN_SLIDE_TIME) {
-			throw new IllegalArgumentException("Window length must be at least " + MIN_SLIDE_TIME + " msecs");
-		}
-		if (windowSlide < MIN_SLIDE_TIME) {
-			throw new IllegalArgumentException("Window slide must be at least " + MIN_SLIDE_TIME + " msecs");
-		}
-		if (windowLength < windowSlide) {
-			throw new IllegalArgumentException("The window size must be larger than the window slide");
-		}
-		
-		final long paneSlide = ArithmeticUtils.gcd(windowLength, windowSlide);
-		if (paneSlide < MIN_SLIDE_TIME) {
-			throw new IllegalArgumentException(String.format(
-					"Cannot compute window of size %d msecs sliding by %d msecs. " +
-							"The unit of grouping is too small: %d msecs", windowLength, windowSlide, paneSlide));
-		}
-		
-		this.function = requireNonNull(function);
-		this.keySelector = requireNonNull(keySelector);
-		this.keySerializer = requireNonNull(keySerializer);
-		this.stateTypeSerializer = requireNonNull(stateTypeSerializer);
-		this.windowSize = windowLength;
-		this.windowSlide = windowSlide;
-		this.paneSize = paneSlide;
-		this.numPanesPerWindow = MathUtils.checkedDownCast(windowLength / paneSlide);
-	}
-	
-	
-	protected abstract AbstractKeyedTimePanes<IN, KEY, STATE, OUT> createPanes(
-			KeySelector<IN, KEY> keySelector, Function function);
-
-	// ------------------------------------------------------------------------
-	//  startup and shutdown
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-
-		out = new TimestampedCollector<>(output);
-		
-		// decide when to first compute the window and when to slide it
-		// the values should align with the start of time (that is, the UNIX epoch, not the big bang)
-		final long now = System.currentTimeMillis();
-		nextEvaluationTime = now + windowSlide - (now % windowSlide);
-		nextSlideTime = now + paneSize - (now % paneSize);
-
-		final long firstTriggerTime = Math.min(nextEvaluationTime, nextSlideTime);
-		
-		// check if we restored state and if we need to fire some windows based on that restored state
-		if (restoredState == null) {
-			// initial empty state: create empty panes that gather the elements per slide
-			panes = createPanes(keySelector, function);
-		}
-		else {
-			// restored state
-			panes = restoredState.panes;
-			
-			long nextPastEvaluationTime = restoredState.nextEvaluationTime;
-			long nextPastSlideTime = restoredState.nextSlideTime;
-			long nextPastTriggerTime = Math.min(nextPastEvaluationTime, nextPastSlideTime);
-			int numPanesRestored = panes.getNumPanes();
-			
-			// fire windows from the past as long as there are more panes with data and as long
-			// as the missed trigger times have not caught up with the presence
-			while (numPanesRestored > 0 && nextPastTriggerTime < firstTriggerTime) {
-				// evaluate the window from the past
-				if (nextPastTriggerTime == nextPastEvaluationTime) {
-					computeWindow(nextPastTriggerTime);
-					nextPastEvaluationTime += windowSlide;
-				}
-				
-				// evaluate slide from the past
-				if (nextPastTriggerTime == nextPastSlideTime) {
-					panes.slidePanes(numPanesPerWindow);
-					numPanesRestored--;
-					nextPastSlideTime += paneSize;
-				}
-
-				nextPastTriggerTime = Math.min(nextPastEvaluationTime, nextPastSlideTime);
-			}
-		}
-		
-		// make sure the first window happens
-		registerTimer(firstTriggerTime, this);
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		
-		final long finalWindowTimestamp = nextEvaluationTime;
-
-		// early stop the triggering thread, so it does not attempt to return any more data
-		stopTriggers();
-
-		// emit the remaining data
-		computeWindow(finalWindowTimestamp);
-	}
-
-	@Override
-	public void dispose() {
-		super.dispose();
-		
-		// acquire the lock during shutdown, to prevent trigger calls at the same time
-		// fail-safe stop of the triggering thread (in case of an error)
-		stopTriggers();
-
-		// release the panes. panes may still be null if dispose is called
-		// after open() failed
-		if (panes != null) {
-			panes.dispose();
-		}
-	}
-	
-	private void stopTriggers() {
-		// reset the action timestamps. this makes sure any pending triggers will not evaluate
-		nextEvaluationTime = -1L;
-		nextSlideTime = -1L;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Receiving elements and triggers
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public void processElement(StreamRecord<IN> element) throws Exception {
-		panes.addElementToLatestPane(element.getValue());
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) {
-		// this operator does not react to watermarks
-	}
-
-	@Override
-	public void trigger(long timestamp) throws Exception {
-		// first we check if we actually trigger the window function
-		if (timestamp == nextEvaluationTime) {
-			// compute and output the results
-			computeWindow(timestamp);
-
-			nextEvaluationTime += windowSlide;
-		}
-
-		// check if we slide the panes by one. this may happen in addition to the
-		// window computation, or just by itself
-		if (timestamp == nextSlideTime) {
-			panes.slidePanes(numPanesPerWindow);
-			nextSlideTime += paneSize;
-		}
-
-		long nextTriggerTime = Math.min(nextEvaluationTime, nextSlideTime);
-		registerTimer(nextTriggerTime, this);
-	}
-	
-	private void computeWindow(long timestamp) throws Exception {
-		out.setTimestamp(timestamp);
-		panes.truncatePanes(numPanesPerWindow);
-		panes.evaluateWindow(out, new TimeWindow(timestamp, timestamp + windowSize));
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpointing
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
-		StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
-		
-		// we write the panes with the key/value maps into the stream, as well as when this state
-		// should have triggered and slided
-		StateBackend.CheckpointStateOutputView out = 
-				getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
-
-		out.writeLong(nextEvaluationTime);
-		out.writeLong(nextSlideTime);
-		panes.writeToOutput(out, keySerializer, stateTypeSerializer);
-		
-		taskState.setOperatorState(out.closeAndGetHandle());
-		return taskState;
-	}
-
-	@Override
-	public void restoreState(StreamTaskState taskState) throws Exception {
-		super.restoreState(taskState);
-
-		@SuppressWarnings("unchecked")
-		StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
-		DataInputView in = inputState.getState(getUserCodeClassloader());
-		
-		final long nextEvaluationTime = in.readLong();
-		final long nextSlideTime = in.readLong();
-
-		AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes = createPanes(keySelector, function);
-		panes.readFromInput(in, keySerializer, stateTypeSerializer);
-		
-		restoredState = new RestoredState<>(panes, nextEvaluationTime, nextSlideTime);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Property access (for testing)
-	// ------------------------------------------------------------------------
-
-	public long getWindowSize() {
-		return windowSize;
-	}
-
-	public long getWindowSlide() {
-		return windowSlide;
-	}
-
-	public long getPaneSize() {
-		return paneSize;
-	}
-	
-	public int getNumPanesPerWindow() {
-		return numPanesPerWindow;
-	}
-
-	public long getNextEvaluationTime() {
-		return nextEvaluationTime;
-	}
-
-	public long getNextSlideTime() {
-		return nextSlideTime;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return "Window (processing time) (length=" + windowSize + ", slide=" + windowSlide + ')';
-	}
-
-	// ------------------------------------------------------------------------
-	// ------------------------------------------------------------------------
-	
-	private static final class RestoredState<IN, KEY, STATE, OUT> {
-
-		final AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes;
-		final long nextEvaluationTime;
-		final long nextSlideTime;
-
-		RestoredState(AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes, long nextEvaluationTime, long nextSlideTime) {
-			this.panes = panes;
-			this.nextEvaluationTime = nextEvaluationTime;
-			this.nextSlideTime = nextSlideTime;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java
deleted file mode 100644
index d1cea20..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.util.Collector;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-
-/**
- * Base class for a multiple key/value maps organized in panes.
- */
-public abstract class AbstractKeyedTimePanes<Type, Key, Aggregate, Result> {
-	
-	private static final int BEGIN_OF_STATE_MAGIC_NUMBER = 0x0FF1CE42;
-
-	private static final int BEGIN_OF_PANE_MAGIC_NUMBER = 0xBADF00D5;
-	
-	/** The latest time pane */
-	protected KeyMap<Key, Aggregate> latestPane = new KeyMap<>();
-
-	/** The previous time panes, ordered by time (early to late) */
-	protected final ArrayDeque<KeyMap<Key, Aggregate>> previousPanes = new ArrayDeque<>();
-
-	// ------------------------------------------------------------------------
-
-	public abstract void addElementToLatestPane(Type element) throws Exception;
-
-	public abstract void evaluateWindow(Collector<Result> out, TimeWindow window) throws Exception;
-	
-	
-	public void dispose() {
-		// since all is heap data, there is no need to clean up anything
-		latestPane = null;
-		previousPanes.clear();
-	}
-	
-	public int getNumPanes() {
-		return previousPanes.size() + 1;
-	}
-	
-	
-	public void slidePanes(int panesToKeep) {
-		if (panesToKeep > 1) {
-			// the current pane becomes the latest previous pane
-			previousPanes.addLast(latestPane);
-
-			// truncate the history
-			while (previousPanes.size() >= panesToKeep) {
-				previousPanes.removeFirst();
-			}
-		}
-
-		// we need a new latest pane
-		latestPane = new KeyMap<>();
-	}
-	
-	public void truncatePanes(int numToRetain) {
-		while (previousPanes.size() >= numToRetain) {
-			previousPanes.removeFirst();
-		}
-	}
-	
-	protected void traverseAllPanes(KeyMap.TraversalEvaluator<Key, Aggregate> traversal, long traversalPass) throws Exception{
-		// gather all panes in an array (faster iterations)
-		@SuppressWarnings({"unchecked", "rawtypes"})
-		KeyMap<Key, Aggregate>[] panes = previousPanes.toArray(new KeyMap[previousPanes.size() + 1]);
-		panes[panes.length - 1] = latestPane;
-
-		// let the maps make a coordinated traversal and evaluate the window function per contained key
-		KeyMap.traverseMaps(panes, traversal, traversalPass);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Serialization and de-serialization
-	// ------------------------------------------------------------------------
-
-	public void writeToOutput(
-			final DataOutputView output,
-			final TypeSerializer<Key> keySerializer,
-			final TypeSerializer<Aggregate> aggSerializer) throws IOException
-	{
-		output.writeInt(BEGIN_OF_STATE_MAGIC_NUMBER);
-		
-		int numPanes = getNumPanes();
-		output.writeInt(numPanes);
-		
-		// write from the past
-		Iterator<KeyMap<Key, Aggregate>> previous = previousPanes.iterator();
-		for (int paneNum = 0; paneNum < numPanes; paneNum++) {
-			output.writeInt(BEGIN_OF_PANE_MAGIC_NUMBER);
-			KeyMap<Key, Aggregate> pane = (paneNum == numPanes - 1) ? latestPane : previous.next();
-			
-			output.writeInt(pane.size());
-			for (KeyMap.Entry<Key, Aggregate> entry : pane) {
-				keySerializer.serialize(entry.getKey(), output);
-				aggSerializer.serialize(entry.getValue(), output);
-			}
-		}
-	}
-	
-	public void readFromInput(
-			final DataInputView input,
-			final TypeSerializer<Key> keySerializer,
-			final TypeSerializer<Aggregate> aggSerializer) throws IOException
-	{
-		validateMagicNumber(BEGIN_OF_STATE_MAGIC_NUMBER, input.readInt());
-		int numPanes = input.readInt();
-		
-		// read from the past towards the presence
-		while (numPanes > 0) {
-			validateMagicNumber(BEGIN_OF_PANE_MAGIC_NUMBER, input.readInt());
-			KeyMap<Key, Aggregate> pane = (numPanes == 1) ? latestPane : new KeyMap<Key, Aggregate>();
-			
-			final int numElementsInPane = input.readInt();
-			for (int i = numElementsInPane - 1; i >= 0; i--) {
-				Key k = keySerializer.deserialize(input);
-				Aggregate a = aggSerializer.deserialize(input);
-				pane.put(k, a);
-			}
-			
-			if (numPanes > 1) {
-				previousPanes.addLast(pane);
-			}
-			numPanes--;
-		}
-	}
-	
-	private static void validateMagicNumber(int expected, int found) throws IOException {
-		if (expected != found) {
-			throw new IOException("Corrupt state stream - wrong magic number. " +
-				"Expected '" + Integer.toHexString(expected) +
-				"', found '" + Integer.toHexString(found) + '\'');
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
deleted file mode 100644
index c854e6c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
+++ /dev/null
@@ -1,133 +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.java.functions.KeySelector;
-import org.apache.flink.runtime.util.UnionIterator;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-import java.util.ArrayList;
-
-
-public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyedTimePanes<Type, Key, ArrayList<Type>, Result> {
-	
-	private final KeySelector<Type, Key> keySelector;
-
-	private final KeyMap.LazyFactory<ArrayList<Type>> listFactory = getListFactory();
-
-	private final WindowFunction<Type, Result, Key, Window> function;
-
-	/**
-	 * IMPORTANT: This value needs to start at one, so it is fresher than the value that new entries have (zero) */
-	private long evaluationPass = 1L;   
-
-	// ------------------------------------------------------------------------
-	
-	public AccumulatingKeyedTimePanes(KeySelector<Type, Key> keySelector, WindowFunction<Type, Result, Key, Window> function) {
-		this.keySelector = keySelector;
-		this.function = function;
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void addElementToLatestPane(Type element) throws Exception {
-		Key k = keySelector.getKey(element);
-		ArrayList<Type> elements = latestPane.putIfAbsent(k, listFactory);
-		elements.add(element);
-	}
-
-	@Override
-	public void evaluateWindow(Collector<Result> out, TimeWindow window) throws Exception {
-		if (previousPanes.isEmpty()) {
-			// optimized path for single pane case (tumbling window)
-			for (KeyMap.Entry<Key, ArrayList<Type>> entry : latestPane) {
-				function.apply(entry.getKey(), window, entry.getValue(), out);
-			}
-		}
-		else {
-			// general code path for multi-pane case
-			WindowFunctionTraversal<Key, Type, Result> evaluator = new WindowFunctionTraversal<>(function, window, out);
-			traverseAllPanes(evaluator, evaluationPass);
-		}
-		
-		evaluationPass++;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Running a window function in a map traversal
-	// ------------------------------------------------------------------------
-	
-	static final class WindowFunctionTraversal<Key, Type, Result> implements KeyMap.TraversalEvaluator<Key, ArrayList<Type>> {
-
-		private final WindowFunction<Type, Result, Key, Window> function;
-		
-		private final UnionIterator<Type> unionIterator;
-		
-		private final Collector<Result> out;
-		
-		private Key currentKey;
-
-		private TimeWindow window;
-
-		WindowFunctionTraversal(WindowFunction<Type, Result, Key, Window> function, TimeWindow window, Collector<Result> out) {
-			this.function = function;
-			this.out = out;
-			this.unionIterator = new UnionIterator<>();
-			this.window = window;
-		}
-
-
-		@Override
-		public void startNewKey(Key key) {
-			unionIterator.clear();
-			currentKey = key;
-		}
-
-		@Override
-		public void nextValue(ArrayList<Type> value) {
-			unionIterator.addList(value);
-		}
-
-		@Override
-		public void keyDone() throws Exception {
-			function.apply(currentKey, window, unionIterator, out);
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Lazy factory for lists (put if absent)
-	// ------------------------------------------------------------------------
-	
-	@SuppressWarnings("unchecked")
-	private static <V> KeyMap.LazyFactory<ArrayList<V>> getListFactory() {
-		return (KeyMap.LazyFactory<ArrayList<V>>) LIST_FACTORY;
-	}
-
-	private static final KeyMap.LazyFactory<?> LIST_FACTORY = new KeyMap.LazyFactory<ArrayList<?>>() {
-
-		@Override
-		public ArrayList<?> create() {
-			return new ArrayList<>(4);
-		}
-	};
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
deleted file mode 100644
index 7a7d04c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
+++ /dev/null
@@ -1,163 +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.Function;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-
-public class AccumulatingProcessingTimeWindowOperator<KEY, IN, OUT> 
-		extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, ArrayList<IN>, WindowFunction<IN, OUT, KEY, TimeWindow>> {
-
-	private static final long serialVersionUID = 7305948082830843475L;
-
-	
-	public AccumulatingProcessingTimeWindowOperator(
-			WindowFunction<IN, OUT, KEY, TimeWindow> function,
-			KeySelector<IN, KEY> keySelector,
-			TypeSerializer<KEY> keySerializer,
-			TypeSerializer<IN> valueSerializer,
-			long windowLength,
-			long windowSlide)
-	{
-		super(function, keySelector, keySerializer,
-				new ArrayListSerializer<IN>(valueSerializer), windowLength, windowSlide);
-	}
-
-	@Override
-	protected AccumulatingKeyedTimePanes<IN, KEY, OUT> createPanes(KeySelector<IN, KEY> keySelector, Function function) {
-		@SuppressWarnings("unchecked")
-		WindowFunction<IN, OUT, KEY, Window> windowFunction = (WindowFunction<IN, OUT, KEY, Window>) function;
-		
-		return new AccumulatingKeyedTimePanes<>(keySelector, windowFunction);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utility Serializer for Lists of Elements
-	// ------------------------------------------------------------------------
-	
-	@SuppressWarnings("ForLoopReplaceableByForEach")
-	private static final class ArrayListSerializer<T> extends TypeSerializer<ArrayList<T>> {
-
-		private static final long serialVersionUID = 1119562170939152304L;
-		
-		private final TypeSerializer<T> elementSerializer;
-
-		ArrayListSerializer(TypeSerializer<T> elementSerializer) {
-			this.elementSerializer = elementSerializer;
-		}
-
-		@Override
-		public boolean isImmutableType() {
-			return false;
-		}
-
-		@Override
-		public TypeSerializer<ArrayList<T>> duplicate() {
-			TypeSerializer<T> duplicateElement = elementSerializer.duplicate();
-			return duplicateElement == elementSerializer ? this : new ArrayListSerializer<T>(duplicateElement);
-		}
-
-		@Override
-		public ArrayList<T> createInstance() {
-			return new ArrayList<>();
-		}
-
-		@Override
-		public ArrayList<T> copy(ArrayList<T> from) {
-			ArrayList<T> newList = new ArrayList<>(from.size());
-			for (int i = 0; i < from.size(); i++) {
-				newList.add(elementSerializer.copy(from.get(i)));
-			}
-			return newList;
-		}
-
-		@Override
-		public ArrayList<T> copy(ArrayList<T> from, ArrayList<T> reuse) {
-			return copy(from);
-		}
-
-		@Override
-		public int getLength() {
-			return -1; // var length
-		}
-
-		@Override
-		public void serialize(ArrayList<T> list, DataOutputView target) throws IOException {
-			final int size = list.size();
-			target.writeInt(size);
-			for (int i = 0; i < size; i++) {
-				elementSerializer.serialize(list.get(i), target);
-			}
-		}
-
-		@Override
-		public ArrayList<T> deserialize(DataInputView source) throws IOException {
-			final int size = source.readInt();
-			final ArrayList<T> list = new ArrayList<>(size);
-			for (int i = 0; i < size; i++) {
-				list.add(elementSerializer.deserialize(source));
-			}
-			return list;
-		}
-
-		@Override
-		public ArrayList<T> deserialize(ArrayList<T> reuse, DataInputView source) throws IOException {
-			return deserialize(source);
-		}
-
-		@Override
-		public void copy(DataInputView source, DataOutputView target) throws IOException {
-			// copy number of elements
-			final int num = source.readInt();
-			target.writeInt(num);
-			for (int i = 0; i < num; i++) {
-				elementSerializer.copy(source, target);
-			}
-		}
-
-		// --------------------------------------------------------------------
-		
-		@Override
-		public boolean equals(Object obj) {
-			return obj == this || 
-					(obj != null && obj.getClass() == getClass() && 
-						elementSerializer.equals(((ArrayListSerializer<?>) obj).elementSerializer));
-		}
-
-		@Override
-		public boolean canEqual(Object obj) {
-			return true;
-		}
-
-		@Override
-		public int hashCode() {
-			return elementSerializer.hashCode();
-		}
-	} 
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java
deleted file mode 100644
index d395b2a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.util.Collector;
-
-
-public class AggregatingKeyedTimePanes<Type, Key> extends AbstractKeyedTimePanes<Type, Key, Type, Type> {
-	
-	private final KeySelector<Type, Key> keySelector;
-	
-	private final ReduceFunction<Type> reducer;
-
-	/**
-	 * IMPORTANT: This value needs to start at one, so it is fresher than the value that new entries have (zero) */
-	private long evaluationPass = 1L;
-
-	// ------------------------------------------------------------------------
-	
-	public AggregatingKeyedTimePanes(KeySelector<Type, Key> keySelector, ReduceFunction<Type> reducer) {
-		this.keySelector = keySelector;
-		this.reducer = reducer;
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void addElementToLatestPane(Type element) throws Exception {
-		Key k = keySelector.getKey(element);
-		latestPane.putOrAggregate(k, element, reducer);
-	}
-
-	@Override
-	public void evaluateWindow(Collector<Type> out, TimeWindow window) throws Exception {
-		if (previousPanes.isEmpty()) {
-			// optimized path for single pane case
-			for (KeyMap.Entry<Key, Type> entry : latestPane) {
-				out.collect(entry.getValue());
-			}
-		}
-		else {
-			// general code path for multi-pane case
-			AggregatingTraversal<Key, Type> evaluator = new AggregatingTraversal<>(reducer, out);
-			traverseAllPanes(evaluator, evaluationPass);
-		}
-		
-		evaluationPass++;
-	}
-
-	// ------------------------------------------------------------------------
-	//  The maps traversal that performs the final aggregation
-	// ------------------------------------------------------------------------
-	
-	static final class AggregatingTraversal<Key, Type> implements KeyMap.TraversalEvaluator<Key, Type> {
-
-		private final ReduceFunction<Type> function;
-		
-		private final Collector<Type> out;
-		
-		private Type currentValue;
-
-		AggregatingTraversal(ReduceFunction<Type> function, Collector<Type> out) {
-			this.function = function;
-			this.out = out;
-		}
-
-		@Override
-		public void startNewKey(Key key) {
-			currentValue = null;
-		}
-
-		@Override
-		public void nextValue(Type value) throws Exception {
-			if (currentValue != null) {
-				currentValue = function.reduce(currentValue, value);
-			}
-			else {
-				currentValue = value;
-			}
-		}
-
-		@Override
-		public void keyDone() throws Exception {
-			out.collect(currentValue);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
deleted file mode 100644
index 0e07cea..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-
-public class AggregatingProcessingTimeWindowOperator<KEY, IN> 
-		extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, IN, IN, ReduceFunction<IN>> {
-
-	private static final long serialVersionUID = 7305948082830843475L;
-
-	
-	public AggregatingProcessingTimeWindowOperator(
-			ReduceFunction<IN> function,
-			KeySelector<IN, KEY> keySelector,
-			TypeSerializer<KEY> keySerializer,
-			TypeSerializer<IN> aggregateSerializer,
-			long windowLength,
-			long windowSlide)
-	{
-		super(function, keySelector, keySerializer, aggregateSerializer, windowLength, windowSlide);
-	}
-
-	@Override
-	protected AggregatingKeyedTimePanes<IN, KEY> createPanes(KeySelector<IN, KEY> keySelector, Function function) {
-		@SuppressWarnings("unchecked")
-		ReduceFunction<IN> windowFunction = (ReduceFunction<IN>) function;
-		
-		return new AggregatingKeyedTimePanes<IN, KEY>(keySelector, windowFunction);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 1bb451a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperator.java
+++ /dev/null
@@ -1,92 +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 com.google.common.annotations.VisibleForTesting;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-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.WindowBufferFactory;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Evicting window operator for non-keyed windows.
- *
- * @see org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator
- *
- * @param <IN> The type of the incoming elements.
- * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
- * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
- */
-public class EvictingNonKeyedWindowOperator<IN, OUT, W extends Window> extends NonKeyedWindowOperator<IN, OUT, W> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final Evictor<? super IN, ? super W> evictor;
-
-	public EvictingNonKeyedWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
-			TypeSerializer<W> windowSerializer,
-			WindowBufferFactory<? super IN, ? extends EvictingWindowBuffer<IN>> windowBufferFactory,
-			AllWindowFunction<IN, OUT, W> windowFunction,
-			Trigger<? super IN, ? super W> trigger,
-			Evictor<? super IN, ? super W> evictor) {
-		super(windowAssigner, windowSerializer, windowBufferFactory, windowFunction, trigger);
-		this.evictor = requireNonNull(evictor);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked, rawtypes")
-	protected void emitWindow(Context context) throws Exception {
-		timestampedCollector.setTimestamp(context.window.maxTimestamp());
-		EvictingWindowBuffer<IN> windowBuffer = (EvictingWindowBuffer<IN>) context.windowBuffer;
-
-		int toEvict = 0;
-		if (windowBuffer.size() > 0) {
-			// need some type trickery here...
-			toEvict = evictor.evict((Iterable) windowBuffer.getElements(), windowBuffer.size(), context.window);
-		}
-
-		windowBuffer.removeElements(toEvict);
-
-		userFunction.apply(
-				context.window,
-				context.windowBuffer.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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
deleted file mode 100644
index ad43812..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-* http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.EvictingWindowBuffer;
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
-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 static java.util.Objects.requireNonNull;
-
-/**
- * A {@link WindowOperator} that also allows an {@link Evictor} to be used.
- *
- * <p>
- * The {@code Evictor} is used to evict elements from panes before processing a window and after
- * a {@link Trigger} has fired.
- *
- * @param <K> The type of key returned by the {@code KeySelector}.
- * @param <IN> The type of the incoming elements.
- * @param <OUT> The type of elements emitted by the {@code WindowFunction}.
- * @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
- */
-public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends WindowOperator<K, IN, OUT, W> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final Evictor<? super IN, ? super W> evictor;
-
-	public EvictingWindowOperator(WindowAssigner<? super IN, W> windowAssigner,
-			TypeSerializer<W> windowSerializer,
-			KeySelector<IN, K> keySelector,
-			TypeSerializer<K> keySerializer,
-			WindowBufferFactory<? super IN, ? extends EvictingWindowBuffer<IN>> windowBufferFactory,
-			WindowFunction<IN, OUT, K, W> windowFunction,
-			Trigger<? super IN, ? super W> trigger,
-			Evictor<? super IN, ? super W> evictor) {
-		super(windowAssigner, windowSerializer, keySelector, keySerializer, windowBufferFactory, windowFunction, trigger);
-		this.evictor = requireNonNull(evictor);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked, rawtypes")
-	protected void emitWindow(Context context) throws Exception {
-		timestampedCollector.setTimestamp(context.window.maxTimestamp());
-		EvictingWindowBuffer<IN> windowBuffer = (EvictingWindowBuffer<IN>) context.windowBuffer;
-
-		int toEvict = 0;
-		if (windowBuffer.size() > 0) {
-			// need some type trickery here...
-			toEvict = evictor.evict((Iterable) windowBuffer.getElements(), windowBuffer.size(), context.window);
-		}
-
-		windowBuffer.removeElements(toEvict);
-
-		userFunction.apply(context.key,
-				context.window,
-				context.windowBuffer.getUnpackedElements(),
-				timestampedCollector);
-	}
-
-	@Override
-	public EvictingWindowOperator<K, 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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMap.java
deleted file mode 100644
index 3f44c4a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMap.java
+++ /dev/null
@@ -1,651 +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.ReduceFunction;
-import org.apache.flink.runtime.util.MathUtils;
-
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-/**
- * A special Hash Map implementation that can be traversed efficiently in sync with other
- * hash maps.
- * <p>
- * The differences between this hash map and Java's "java.util.HashMap" are:
- * <ul>
- *     <li>A different hashing scheme. This implementation uses extensible hashing, meaning that
- *         each hash table growth takes one more lower hash code bit into account, and values that where
- *         formerly in the same bucket will afterwards be in the two adjacent buckets.</li>
- *     <li>This allows an efficient traversal of multiple hash maps together, even though the maps are
- *         of different sizes.</li>
- *     <li>The map offers functions such as "putIfAbsent()" and "putOrAggregate()"</li>
- *     <li>The map supports no removal/shrinking.</li>
- * </ul>
- */
-public class KeyMap<K, V> implements Iterable<KeyMap.Entry<K, V>> {
-	
-	/** The minimum table capacity, 64 entries */
-	private static final int MIN_CAPACITY = 0x40;
-	
-	/** The maximum possible table capacity, the largest positive power of
-	 * two in the 32bit signed integer value range */
-	private static final int MAX_CAPACITY = 0x40000000;
-	
-	/** The number of bits used for table addressing when table is at max capacity */
-	private static final int FULL_BIT_RANGE = MathUtils.log2strict(MAX_CAPACITY);
-	
-	// ------------------------------------------------------------------------
-	
-	/** The hash index, as an array of entries */
-	private Entry<K, V>[] table;
-	
-	/** The number of bits by which the hash code is shifted right, to find the bucket */
-	private int shift;
-	
-	/** The number of elements in the hash table */
-	private int numElements;
-	
-	/** The number of elements above which the hash table needs to grow */
-	private int rehashThreshold;
-	
-	/** The base-2 logarithm of the table capacity */ 
-	private int log2size;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new hash table with the default initial capacity.
-	 */
-	public KeyMap() {
-		this(0);
-	}
-
-	/**
-	 * Creates a new table with a capacity tailored to the given expected number of elements.
-	 * 
-	 * @param expectedNumberOfElements The number of elements to tailor the capacity to.
-	 */
-	public KeyMap(int expectedNumberOfElements) {
-		if (expectedNumberOfElements < 0) {
-			throw new IllegalArgumentException("Invalid capacity: " + expectedNumberOfElements);
-		}
-		
-		// round up to the next power or two
-		// guard against too small capacity and integer overflows
-		int capacity = Integer.highestOneBit(expectedNumberOfElements) << 1;
-		capacity = capacity >= 0 ? Math.max(MIN_CAPACITY, capacity) : MAX_CAPACITY;
-
-		// this also acts as a sanity check
-		log2size = MathUtils.log2strict(capacity);
-		shift = FULL_BIT_RANGE - log2size;
-		table = allocateTable(capacity);
-		rehashThreshold = getRehashThreshold(capacity);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Gets and Puts
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Inserts the given value, mapped under the given key. If the table already contains a value for
-	 * the key, the value is replaced and returned. If no value is contained, yet, the function
-	 * returns null.
-	 * 
-	 * @param key The key to insert.
-	 * @param value The value to insert.
-	 * @return The previously mapped value for the key, or null, if no value was mapped for the key.
-	 * 
-	 * @throws java.lang.NullPointerException Thrown, if the key is null.
-	 */
-	public final V put(K key, V value) {
-		final int hash = hash(key);
-		final int slot = indexOf (hash);
-		
-		// search the chain from the slot
-		for (Entry<K, V> e = table[slot]; e != null; e = e.next) {
-			Object k;
-			if (e.hashCode == hash && ((k = e.key) == key || key.equals(k))) {
-				// found match
-				V old = e.value;
-				e.value = value;
-				return old;
-			}
-		}
-
-		// no match, insert a new value
-		insertNewEntry(hash, key, value, slot);
-		return null;
-	}
-
-	/**
-	 * Inserts a value for the given key, if no value is yet contained for that key. Otherwise,
-	 * returns the value currently contained for the key.
-	 * <p>
-	 * The value that is inserted in case that the key is not contained, yet, is lazily created
-	 * using the given factory.
-	 *
-	 * @param key The key to insert.
-	 * @param factory The factory that produces the value, if no value is contained, yet, for the key.
-	 * @return The value in the map after this operation (either the previously contained value, or the
-	 *         newly created value).
-	 * 
-	 * @throws java.lang.NullPointerException Thrown, if the key is null.
-	 */
-	public final V putIfAbsent(K key, LazyFactory<V> factory) {
-		final int hash = hash(key);
-		final int slot = indexOf(hash);
-
-		// search the chain from the slot
-		for (Entry<K, V> entry = table[slot]; entry != null; entry = entry.next) {
-			if (entry.hashCode == hash && entry.key.equals(key)) {
-				// found match
-				return entry.value;
-			}
-		}
-
-		// no match, insert a new value
-		V value = factory.create();
-		insertNewEntry(hash, key, value, slot);
-
-		// return the created value
-		return value;
-	}
-
-	/**
-	 * Inserts or aggregates a value into the hash map. If the hash map does not yet contain the key,
-	 * this method inserts the value. If the table already contains the key (and a value) this
-	 * method will use the given ReduceFunction function to combine the existing value and the
-	 * given value to a new value, and store that value for the key. 
-	 * 
-	 * @param key The key to map the value.
-	 * @param value The new value to insert, or aggregate with the existing value.
-	 * @param aggregator The aggregator to use if a value is already contained.
-	 * 
-	 * @return The value in the map after this operation: Either the given value, or the aggregated value.
-	 * 
-	 * @throws java.lang.NullPointerException Thrown, if the key is null.
-	 * @throws Exception The method forwards exceptions from the aggregation function.
-	 */
-	public final V putOrAggregate(K key, V value, ReduceFunction<V> aggregator) throws Exception {
-		final int hash = hash(key);
-		final int slot = indexOf(hash);
-
-		// search the chain from the slot
-		for (Entry<K, V> entry = table[slot]; entry != null; entry = entry.next) {
-			if (entry.hashCode == hash && entry.key.equals(key)) {
-				// found match
-				entry.value = aggregator.reduce(entry.value, value);
-				return entry.value;
-			}
-		}
-
-		// no match, insert a new value
-		insertNewEntry(hash, key, value, slot);
-		// return the original value
-		return value;
-	}
-
-	/**
-	 * Looks up the value mapped under the given key. Returns null if no value is mapped under this key.
-	 * 
-	 * @param key The key to look up.
-	 * @return The value associated with the key, or null, if no value is found for the key.
-	 * 
-	 * @throws java.lang.NullPointerException Thrown, if the key is null.
-	 */
-	public V get(K key) {
-		final int hash = hash(key);
-		final int slot = indexOf(hash);
-		
-		// search the chain from the slot
-		for (Entry<K, V> entry = table[slot]; entry != null; entry = entry.next) {
-			if (entry.hashCode == hash && entry.key.equals(key)) {
-				return entry.value;
-			}
-		}
-		
-		// not found
-		return null;
-	}
-
-	private void insertNewEntry(int hashCode, K key, V value, int position) {
-		Entry<K,V> e = table[position];
-		table[position] = new Entry<>(key, value, hashCode, e);
-		numElements++;
-
-		// rehash if necessary
-		if (numElements > rehashThreshold) {
-			growTable();
-		}
-	}
-	
-	private int indexOf(int hashCode) {
-		return (hashCode >> shift) & (table.length - 1);
-	}
-
-	/**
-	 * Creates an iterator over the entries of this map.
-	 * 
-	 * @return An iterator over the entries of this map.
-	 */
-	@Override
-	public Iterator<Entry<K, V>> iterator() {
-		return new Iterator<Entry<K, V>>() {
-			
-			private final Entry<K, V>[] tab = KeyMap.this.table;
-			
-			private Entry<K, V> nextEntry;
-			
-			private int nextPos = 0;
-			
-			@Override
-			public boolean hasNext() {
-				if (nextEntry != null) {
-					return true;
-				}
-				else {
-					while (nextPos < tab.length) {
-						Entry<K, V> e = tab[nextPos++];
-						if (e != null) {
-							nextEntry = e;
-							return true;
-						}
-					}
-					return false;
-				}
-			}
-
-			@Override
-			public Entry<K, V> next() {
-				if (nextEntry != null || hasNext()) {
-					Entry<K, V> e = nextEntry;
-					nextEntry = nextEntry.next;
-					return e;
-				}
-				else {
-					throw new NoSuchElementException();
-				}
-			}
-
-			@Override
-			public void remove() {
-				throw new UnsupportedOperationException();
-			}
-		};
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Gets the number of elements currently in the map.
-	 * @return The number of elements currently in the map.
-	 */
-	public int size() {
-		return numElements;
-	}
-
-	/**
-	 * Checks whether the map is empty.
-	 * @return True, if the map is empty, false otherwise.
-	 */
-	public boolean isEmpty() {
-		return numElements == 0;
-	}
-
-	/**
-	 * Gets the current table capacity, i.e., the number of slots in the hash table, without
-	 * and overflow chaining.
-	 * @return The number of slots in the hash table.
-	 */
-	public int getCurrentTableCapacity() {
-		return table.length;
-	}
-
-	/**
-	 * Gets the base-2 logarithm of the hash table capacity, as returned by
-	 * {@link #getCurrentTableCapacity()}.
-	 * 
-	 * @return The base-2 logarithm of the hash table capacity.
-	 */
-	public int getLog2TableCapacity() {
-		return log2size;
-	}
-	
-	public int getRehashThreshold() {
-		return rehashThreshold;
-	}
-	
-	public int getShift() {
-		return shift;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-	
-	@SuppressWarnings("unchecked")
-	private Entry<K, V>[] allocateTable(int numElements) {
-		return (Entry<K, V>[]) new Entry<?, ?>[numElements];
-	}
-	
-	private void growTable() {
-		final int newSize = table.length << 1;
-				
-		// only grow if there is still space to grow the table
-		if (newSize > 0) {
-			final Entry<K, V>[] oldTable = this.table;
-			final Entry<K, V>[] newTable = allocateTable(newSize);
-
-			final int newShift = shift - 1;
-			final int newMask = newSize - 1;
-			
-			// go over all slots from the table. since we hash to adjacent positions in
-			// the new hash table, this is actually cache efficient
-			for (Entry<K, V> entry : oldTable) {
-				// traverse the chain for each slot
-				while (entry != null) {
-					final int newPos = (entry.hashCode >> newShift) & newMask;
-					Entry<K, V> nextEntry = entry.next;
-					entry.next = newTable[newPos];
-					newTable[newPos] = entry;
-					entry = nextEntry;
-				}
-			}
-			
-			this.table = newTable;
-			this.shift = newShift;
-			this.rehashThreshold = getRehashThreshold(newSize);
-			this.log2size += 1;
-		}
-	}
-	
-	private static int hash(Object key) {
-		int code = key.hashCode();
-		
-		// we need a strong hash function that generates diverse upper bits
-		// this hash function is more expensive than the "scramble" used by "java.util.HashMap",
-		// but required for this sort of hash table
-		code = (code + 0x7ed55d16) + (code << 12);
-		code = (code ^ 0xc761c23c) ^ (code >>> 19);
-		code = (code + 0x165667b1) + (code << 5);
-		code = (code + 0xd3a2646c) ^ (code << 9);
-		code = (code + 0xfd7046c5) + (code << 3);
-		return (code ^ 0xb55a4f09) ^ (code >>> 16);
-	}
-	
-	private static int getRehashThreshold(int capacity) {
-		// divide before multiply, to avoid overflow
-		return capacity / 4 * 3;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Testing Utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * For testing only: Actively counts the number of entries, rather than using the
-	 * counter variable. This method has linear complexity, rather than constant.
-	 * 
-	 * @return The counted number of entries.
-	 */
-	int traverseAndCountElements() {
-		int num = 0;
-		
-		for (Entry<?, ?> entry : table) {
-			while (entry != null) {
-				num++;
-				entry = entry.next;
-			}
-		}
-		
-		return num;
-	}
-
-	/**
-	 * For testing only: Gets the length of the longest overflow chain.
-	 * This method has linear complexity.
-	 * 
-	 * @return The length of the longest overflow chain.
-	 */
-	int getLongestChainLength() {
-		int maxLen = 0;
-
-		for (Entry<?, ?> entry : table) {
-			int thisLen = 0;
-			while (entry != null) {
-				thisLen++;
-				entry = entry.next;
-			}
-			maxLen = Math.max(maxLen, thisLen);
-		}
-
-		return maxLen;
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * An entry in the hash table.
-	 * 
-	 * @param <K> Type of the key.
-	 * @param <V> Type of the value.
-	 */
-	public static final class Entry<K, V> {
-		
-		final K key;
-		final int hashCode;
-		
-		V value;
-		Entry<K, V> next;
-		long touchedTag;
-
-		Entry(K key, V value, int hashCode, Entry<K, V> next) {
-			this.key = key;
-			this.value = value;
-			this.next = next;
-			this.hashCode = hashCode;
-		}
-
-		public K getKey() {
-			return key;
-		}
-
-		public V getValue() {
-			return value;
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Performs a traversal about logical the multi-map that results from the union of the
-	 * given maps. This method does not actually build a union of the map, but traverses the hash maps
-	 * together.
-	 * 
-	 * @param maps The array uf maps whose union should be traversed.
-	 * @param visitor The visitor that is called for each key and all values.
-	 * @param touchedTag A tag that is used to mark elements that have been touched in this specific
-	 *                   traversal. Each successive traversal should supply a larger value for this
-	 *                   tag than the previous one.
-	 * 
-	 * @param <K> The type of the map's key.
-	 * @param <V> The type of the map's value.
-	 */
-	public static <K, V> void traverseMaps(
-					final KeyMap<K, V>[] maps,
-					final TraversalEvaluator<K, V> visitor,
-					final long touchedTag)
-		throws Exception
-	{
-		// we need to work on the maps in descending size
-		Arrays.sort(maps, CapacityDescendingComparator.INSTANCE);
-		
-		final int[] shifts = new int[maps.length];
-		final int[] lowBitsMask = new int[maps.length];
-		final int numSlots = maps[0].table.length;
-		final int numTables = maps.length;
-		
-		// figure out how much each hash table collapses the entries
-		for (int i = 0; i < numTables; i++) {
-			shifts[i] = maps[0].log2size - maps[i].log2size;
-			lowBitsMask[i] = (1 << shifts[i]) - 1;
-		}
-		
-		// go over all slots (based on the largest hash table)
-		for (int pos = 0; pos < numSlots; pos++) {
-			
-			// for each slot, go over all tables, until the table does not have that slot any more
-			// for tables where multiple slots collapse into one, we visit that one when we process the
-			// latest of all slots that collapse to that one
-			int mask;
-			for (int rootTable = 0;
-					rootTable < numTables && ((mask = lowBitsMask[rootTable]) & pos) == mask;
-					rootTable++)
-			{
-				// use that table to gather keys and start collecting keys from the following tables
-				// go over all entries of that slot in the table
-				Entry<K, V> entry = maps[rootTable].table[pos >> shifts[rootTable]];
-				while (entry != null) {
-					// take only entries that have not been collected as part of other tables
-					if (entry.touchedTag < touchedTag) {
-						entry.touchedTag = touchedTag;
-						
-						final K key = entry.key;
-						final int hashCode = entry.hashCode;
-						visitor.startNewKey(key);
-						visitor.nextValue(entry.value);
-						
-						addEntriesFromChain(entry.next, visitor, key, touchedTag, hashCode);
-						
-						// go over the other hash tables and collect their entries for the key
-						for (int followupTable = rootTable + 1; followupTable < numTables; followupTable++) {
-							Entry<K, V> followupEntry = maps[followupTable].table[pos >> shifts[followupTable]];
-							if (followupEntry != null) {
-								addEntriesFromChain(followupEntry, visitor, key, touchedTag, hashCode);
-							}
-						}
-
-						visitor.keyDone();
-					}
-					
-					entry = entry.next;
-				}
-			}
-		}
-	}
-	
-	private static <K, V> void addEntriesFromChain(
-			Entry<K, V> entry,
-			TraversalEvaluator<K, V> visitor,
-			K key,
-			long touchedTag,
-			int hashCode) throws Exception
-	{
-		while (entry != null) {
-			if (entry.touchedTag < touchedTag && entry.hashCode == hashCode && entry.key.equals(key)) {
-				entry.touchedTag = touchedTag;
-				visitor.nextValue(entry.value);
-			}
-			entry = entry.next;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Comparator that defines a descending order on maps depending on their table capacity
-	 * and number of elements.
-	 */
-	static final class CapacityDescendingComparator implements Comparator<KeyMap<?, ?>> {
-		
-		static final CapacityDescendingComparator INSTANCE = new CapacityDescendingComparator();
-		
-		private CapacityDescendingComparator() {}
-
-
-		@Override
-		public int compare(KeyMap<?, ?> o1, KeyMap<?, ?> o2) {
-			// this sorts descending
-			int cmp = o2.getLog2TableCapacity() - o1.getLog2TableCapacity();
-			if (cmp != 0) {
-				return cmp;
-			}
-			else {
-				return o2.size() - o1.size();
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A factory for lazy/on-demand instantiation of values.
-	 *
-	 * @param <V> The type created by the factory.
-	 */
-	public static interface LazyFactory<V> {
-
-		/**
-		 * The factory method; creates the value.
-		 * @return The value.
-		 */
-		V create();
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A visitor for a traversal over the union of multiple hash maps. The visitor is
-	 * called for each key in the union of the maps and all values associated with that key
-	 * (one per map, but multiple across maps). 
-	 * 
-	 * @param <K> The type of the key.
-	 * @param <V> The type of the value.
-	 */
-	public static interface TraversalEvaluator<K, V> {
-
-		/**
-		 * Called whenever the traversal starts with a new key.
-		 * 
-		 * @param key The key traversed.
-		 * @throws Exception Method forwards all exceptions.
-		 */
-		void startNewKey(K key) throws Exception;
-
-		/**
-		 * Called for each value found for the current key.
-		 * 
-		 * @param value The next value.
-		 * @throws Exception Method forwards all exceptions.
-		 */
-		void nextValue(V value) throws Exception;
-
-		/**
-		 * Called when the traversal for the current key is complete.
-		 * 
-		 * @throws Exception Method forwards all exceptions.
-		 */
-		void keyDone() throws Exception;
-	}
-}


[08/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java
new file mode 100644
index 0000000..987e6c5
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DiscardingSink.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+
+/**
+ * Sink function that discards data.
+ * @param <T> The type of the function.
+ */
+public class DiscardingSink<T> implements SinkFunction<T> {
+
+	private static final long serialVersionUID = 2777597566520109843L;
+
+	@Override
+	public void invoke(T value) {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
new file mode 100644
index 0000000..5a8ffaa
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka.testutils;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class FailingIdentityMapper<T> extends RichMapFunction<T,T> implements
+		Checkpointed<Integer>, CheckpointNotifier, Runnable {
+	
+	private static final Logger LOG = LoggerFactory.getLogger(FailingIdentityMapper.class);
+	
+	private static final long serialVersionUID = 6334389850158707313L;
+	
+	public static volatile boolean failedBefore;
+	public static volatile boolean hasBeenCheckpointedBeforeFailure;
+
+	private final int failCount;
+	private int numElementsTotal;
+	private int numElementsThisTime;
+	
+	private boolean failer;
+	private boolean hasBeenCheckpointed;
+	
+	private Thread printer;
+	private volatile boolean printerRunning = true;
+
+	public FailingIdentityMapper(int failCount) {
+		this.failCount = failCount;
+	}
+
+	@Override
+	public void open(Configuration parameters) {
+		failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
+		printer = new Thread(this, "FailingIdentityMapper Status Printer");
+		printer.start();
+	}
+
+	@Override
+	public T map(T value) throws Exception {
+		numElementsTotal++;
+		numElementsThisTime++;
+		
+		if (!failedBefore) {
+			Thread.sleep(10);
+			
+			if (failer && numElementsTotal >= failCount) {
+				hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed;
+				failedBefore = true;
+				throw new Exception("Artificial Test Failure");
+			}
+		}
+		return value;
+	}
+
+	@Override
+	public void close() throws Exception {
+		printerRunning = false;
+		if (printer != null) {
+			printer.interrupt();
+			printer = null;
+		}
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) {
+		this.hasBeenCheckpointed = true;
+	}
+
+	@Override
+	public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+		return numElementsTotal;
+	}
+
+	@Override
+	public void restoreState(Integer state) {
+		numElementsTotal = state;
+	}
+
+	@Override
+	public void run() {
+		while (printerRunning) {
+			try {
+				Thread.sleep(5000);
+			}
+			catch (InterruptedException e) {
+				// ignore
+			}
+			LOG.info("============================> Failing mapper  {}: count={}, totalCount={}",
+					getRuntimeContext().getIndexOfThisSubtask(),
+					numElementsThisTime, numElementsTotal);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
new file mode 100644
index 0000000..e94adb5
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.client.JobStatusMessage;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class JobManagerCommunicationUtils {
+	
+	private static final FiniteDuration askTimeout = new FiniteDuration(30, TimeUnit.SECONDS);
+	
+	
+	public static void cancelCurrentJob(ActorGateway jobManager) throws Exception {
+		
+		// find the jobID
+		Future<Object> listResponse = jobManager.ask(
+				JobManagerMessages.getRequestRunningJobsStatus(),
+				askTimeout);
+
+		List<JobStatusMessage> jobs;
+		try {
+			Object result = Await.result(listResponse, askTimeout);
+			jobs = ((JobManagerMessages.RunningJobsStatus) result).getStatusMessages();
+		}
+		catch (Exception e) {
+			throw new Exception("Could not cancel job - failed to retrieve running jobs from the JobManager.", e);
+		}
+		
+		if (jobs.isEmpty()) {
+			throw new Exception("Could not cancel job - no running jobs");
+		}
+		if (jobs.size() != 1) {
+			throw new Exception("Could not cancel job - more than one running job.");
+		}
+		
+		JobStatusMessage status = jobs.get(0);
+		if (status.getJobState().isTerminalState()) {
+			throw new Exception("Could not cancel job - job is not running any more");
+		}
+		
+		JobID jobId = status.getJobId();
+		
+		Future<Object> response = jobManager.ask(new JobManagerMessages.CancelJob(jobId), askTimeout);
+		try {
+			Await.result(response, askTimeout);
+		}
+		catch (Exception e) {
+			throw new Exception("Sending the 'cancel' message failed.", e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
new file mode 100644
index 0000000..b9fc3de
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.DoubleCounter;
+import org.apache.flink.api.common.accumulators.Histogram;
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.cache.DistributedCache;
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.OperatorState;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+public class MockRuntimeContext implements RuntimeContext {
+
+	private final int numberOfParallelSubtasks;
+	private final int indexOfThisSubtask;
+
+	public MockRuntimeContext(int numberOfParallelSubtasks, int indexOfThisSubtask) {
+		this.numberOfParallelSubtasks = numberOfParallelSubtasks;
+		this.indexOfThisSubtask = indexOfThisSubtask;
+	}
+
+
+	@Override
+	public String getTaskName() {
+		return null;
+	}
+
+	@Override
+	public int getNumberOfParallelSubtasks() {
+		return numberOfParallelSubtasks;
+	}
+
+	@Override
+	public int getIndexOfThisSubtask() {
+		return indexOfThisSubtask;
+	}
+
+	@Override
+	public ExecutionConfig getExecutionConfig() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public ClassLoader getUserCodeClassLoader() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Map<String, Accumulator<?, ?>> getAllAccumulators() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public IntCounter getIntCounter(String name) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public LongCounter getLongCounter(String name) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public DoubleCounter getDoubleCounter(String name) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public Histogram getHistogram(String name) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public <RT> List<RT> getBroadcastVariable(String name) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public DistributedCache getDistributedCache() {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
new file mode 100644
index 0000000..e105e01
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/PartitionValidatingMapper.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.api.common.functions.MapFunction;
+
+import java.util.HashSet;
+import java.util.Set;
+
+
+public class PartitionValidatingMapper implements MapFunction<Integer, Integer> {
+
+	private static final long serialVersionUID = 1088381231244959088L;
+	
+	/* the partitions from which this function received data */
+	private final Set<Integer> myPartitions = new HashSet<>();
+	
+	private final int numPartitions;
+	private final int maxPartitions;
+
+	public PartitionValidatingMapper(int numPartitions, int maxPartitions) {
+		this.numPartitions = numPartitions;
+		this.maxPartitions = maxPartitions;
+	}
+
+	@Override
+	public Integer map(Integer value) throws Exception {
+		// validate that the partitioning is identical
+		int partition = value % numPartitions;
+		myPartitions.add(partition);
+		if (myPartitions.size() > maxPartitions) {
+			throw new Exception("Error: Elements from too many different partitions: " + myPartitions
+					+ ". Expect elements only from " + maxPartitions + " partitions");
+		}
+		return value;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java
new file mode 100644
index 0000000..12e3460
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/SuccessException.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+/**
+ * Exception that is thrown to terminate a program and indicate success.
+ */
+public class SuccessException extends Exception {
+	private static final long serialVersionUID = -7011865671593955887L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
new file mode 100644
index 0000000..1d61229
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ThrottledMapper.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.api.common.functions.MapFunction;
+
+/**
+ * An identity map function that sleeps between elements, throttling the
+ * processing speed.
+ * 
+ * @param <T> The type mapped.
+ */
+public class ThrottledMapper<T> implements MapFunction<T,T> {
+
+	private static final long serialVersionUID = 467008933767159126L;
+
+	private final int sleep;
+
+	public ThrottledMapper(int sleep) {
+		this.sleep = sleep;
+	}
+
+	@Override
+	public T map(T value) throws Exception {
+		Thread.sleep(this.sleep);
+		return value;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
new file mode 100644
index 0000000..b762e21
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka.testutils;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
+
+import java.io.Serializable;
+
+/**
+ * Special partitioner that uses the first field of a 2-tuple as the partition,
+ * and that expects a specific number of partitions.
+ */
+public class Tuple2Partitioner extends KafkaPartitioner implements Serializable {
+	
+	private static final long serialVersionUID = 1L;
+
+	private final int expectedPartitions;
+
+	
+	public Tuple2Partitioner(int expectedPartitions) {
+		this.expectedPartitions = expectedPartitions;
+	}
+
+	@Override
+	public int partition(Object key, int numPartitions) {
+		if (numPartitions != expectedPartitions) {
+			throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions");
+		}
+		@SuppressWarnings("unchecked")
+		Tuple2<Integer, Integer> element = (Tuple2<Integer, Integer>) key;
+		
+		return element.f0;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
new file mode 100644
index 0000000..f3cc4fa
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/ValidatingExactlyOnceSink.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.BitSet;
+
+public class ValidatingExactlyOnceSink implements SinkFunction<Integer>, Checkpointed<Tuple2<Integer, BitSet>> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ValidatingExactlyOnceSink.class);
+
+	private static final long serialVersionUID = 1748426382527469932L;
+	
+	private final int numElementsTotal;
+	
+	private BitSet duplicateChecker = new BitSet();  // this is checkpointed
+
+	private int numElements; // this is checkpointed
+
+	
+	public ValidatingExactlyOnceSink(int numElementsTotal) {
+		this.numElementsTotal = numElementsTotal;
+	}
+
+	
+	@Override
+	public void invoke(Integer value) throws Exception {
+		numElements++;
+		
+		if (duplicateChecker.get(value)) {
+			throw new Exception("Received a duplicate");
+		}
+		duplicateChecker.set(value);
+		if (numElements == numElementsTotal) {
+			// validate
+			if (duplicateChecker.cardinality() != numElementsTotal) {
+				throw new Exception("Duplicate checker has wrong cardinality");
+			}
+			else if (duplicateChecker.nextClearBit(0) != numElementsTotal) {
+				throw new Exception("Received sparse sequence");
+			}
+			else {
+				throw new SuccessException();
+			}
+		}
+	}
+
+	@Override
+	public Tuple2<Integer, BitSet> snapshotState(long checkpointId, long checkpointTimestamp) {
+		LOG.info("Snapshot of counter "+numElements+" at checkpoint "+checkpointId);
+		return new Tuple2<>(numElements, duplicateChecker);
+	}
+
+	@Override
+	public void restoreState(Tuple2<Integer, BitSet> state) {
+		LOG.info("restoring num elements to {}", state.f0);
+		this.numElements = state.f0;
+		this.duplicateChecker = state.f1;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..6bdfb48
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/resources/log4j-test.properties
@@ -0,0 +1,29 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+log4j.rootLogger=INFO, testlogger
+
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..45b3b92
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/test/resources/logback-test.xml
@@ -0,0 +1,30 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+    <logger name="org.apache.flink.streaming" level="WARN"/>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/pom.xml b/flink-streaming-connectors/flink-connector-nifi/pom.xml
new file mode 100644
index 0000000..a590b07
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/pom.xml
@@ -0,0 +1,94 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-streaming-connectors-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-nifi</artifactId>
+	<name>flink-connector-nifi</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<nifi.version>0.3.0</nifi.version>
+	</properties>
+
+	<dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-site-to-site-client</artifactId>
+            <version>${nifi.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-tests</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<rerunFailingTestsCount>3</rerunFailingTestsCount>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-failsafe-plugin</artifactId>
+				<configuration>
+					<rerunFailingTestsCount>3</rerunFailingTestsCount>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java
new file mode 100644
index 0000000..c8ceb57
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacket.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.nifi;
+
+import java.util.Map;
+
+/**
+ * <p>
+ * The NiFiDataPacket provides a packaging around a NiFi FlowFile. It wraps both
+ * a FlowFile's content and its attributes so that they can be processed by Flink.
+ * </p>
+ */
+public interface NiFiDataPacket {
+
+	/**
+	 * @return the contents of a NiFi FlowFile
+	 */
+	byte[] getContent();
+
+	/**
+	 * @return a Map of attributes that are associated with the NiFi FlowFile
+	 */
+	Map<String, String> getAttributes();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java
new file mode 100644
index 0000000..9bb521b
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiDataPacketBuilder.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.nifi;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+import java.io.Serializable;
+
+/**
+ * A function that can create a NiFiDataPacket from an incoming instance of the given type.
+ *
+ * @param <T>
+ */
+public interface NiFiDataPacketBuilder<T> extends Function, Serializable {
+
+	NiFiDataPacket createNiFiDataPacket(T t, RuntimeContext ctx);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java
new file mode 100644
index 0000000..abc6b35
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.nifi;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+import org.apache.nifi.remote.client.SiteToSiteClientConfig;
+
+/**
+ * A sink that delivers data to Apache NiFi using the NiFi Site-to-Site client. The sink requires
+ * a NiFiDataPacketBuilder which can create instances of NiFiDataPacket from the incoming data.
+ */
+public class NiFiSink<T> extends RichSinkFunction<T> {
+
+	private SiteToSiteClient client;
+	private SiteToSiteClientConfig clientConfig;
+	private NiFiDataPacketBuilder<T> builder;
+
+	/**
+	 * Construct a new NiFiSink with the given client config and NiFiDataPacketBuilder.
+	 *
+	 * @param clientConfig the configuration for building a NiFi SiteToSiteClient
+	 * @param builder a builder to produce NiFiDataPackets from incoming data
+	 */
+	public NiFiSink(SiteToSiteClientConfig clientConfig, NiFiDataPacketBuilder<T> builder) {
+		this.clientConfig = clientConfig;
+		this.builder = builder;
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		this.client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build();
+	}
+
+	@Override
+	public void invoke(T value) throws Exception {
+		final NiFiDataPacket niFiDataPacket = builder.createNiFiDataPacket(value, getRuntimeContext());
+
+		final Transaction transaction = client.createTransaction(TransferDirection.SEND);
+		if (transaction == null) {
+			throw new IllegalStateException("Unable to create a NiFi Transaction to send data");
+		}
+
+		transaction.send(niFiDataPacket.getContent(), niFiDataPacket.getAttributes());
+		transaction.confirm();
+		transaction.complete();
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		client.close();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java
new file mode 100644
index 0000000..a213bb4
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.nifi;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.nifi.remote.Transaction;
+import org.apache.nifi.remote.TransferDirection;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+import org.apache.nifi.remote.client.SiteToSiteClientConfig;
+import org.apache.nifi.remote.protocol.DataPacket;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A source that pulls data from Apache NiFi using the NiFi Site-to-Site client. This source
+ * produces NiFiDataPackets which encapsulate the content and attributes of a NiFi FlowFile.
+ */
+public class NiFiSource extends RichParallelSourceFunction<NiFiDataPacket> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(NiFiSource.class);
+
+	private static final long DEFAULT_WAIT_TIME_MS = 1000;
+
+	private long waitTimeMs;
+	private SiteToSiteClient client;
+	private SiteToSiteClientConfig clientConfig;
+	private transient volatile boolean running;
+
+	/**
+	 * Constructs a new NiFiSource using the given client config and the default wait time of 1000 ms.
+	 *
+	 * @param clientConfig the configuration for building a NiFi SiteToSiteClient
+	 */
+	public NiFiSource(SiteToSiteClientConfig clientConfig) {
+		this(clientConfig, DEFAULT_WAIT_TIME_MS);
+	}
+
+	/**
+	 * Constructs a new NiFiSource using the given client config and wait time.
+	 *
+	 * @param clientConfig the configuration for building a NiFi SiteToSiteClient
+	 * @param waitTimeMs the amount of time to wait (in milliseconds) if no data is available to pull from NiFi
+	 */
+	public NiFiSource(SiteToSiteClientConfig clientConfig, long waitTimeMs) {
+		this.clientConfig = clientConfig;
+		this.waitTimeMs = waitTimeMs;
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build();
+		running = true;
+	}
+
+	@Override
+	public void run(SourceContext<NiFiDataPacket> ctx) throws Exception {
+		try {
+			while (running) {
+				final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE);
+				if (transaction == null) {
+					LOG.warn("A transaction could not be created, waiting and will try again...");
+					try {
+						Thread.sleep(waitTimeMs);
+					} catch (InterruptedException e) {
+
+					}
+					continue;
+				}
+
+				DataPacket dataPacket = transaction.receive();
+				if (dataPacket == null) {
+					transaction.confirm();
+					transaction.complete();
+
+					LOG.debug("No data available to pull, waiting and will try again...");
+					try {
+						Thread.sleep(waitTimeMs);
+					} catch (InterruptedException e) {
+
+					}
+					continue;
+				}
+
+				final List<NiFiDataPacket> niFiDataPackets = new ArrayList<>();
+				do {
+					// Read the data into a byte array and wrap it along with the attributes
+					// into a NiFiDataPacket.
+					final InputStream inStream = dataPacket.getData();
+					final byte[] data = new byte[(int) dataPacket.getSize()];
+					StreamUtils.fillBuffer(inStream, data);
+
+					final Map<String, String> attributes = dataPacket.getAttributes();
+
+					niFiDataPackets.add(new StandardNiFiDataPacket(data, attributes));
+					dataPacket = transaction.receive();
+				} while (dataPacket != null);
+
+				// Confirm transaction to verify the data
+				transaction.confirm();
+
+				for (NiFiDataPacket dp : niFiDataPackets) {
+					ctx.collect(dp);
+				}
+
+				transaction.complete();
+			}
+		} finally {
+			ctx.close();
+		}
+	}
+
+	@Override
+	public void cancel() {
+		running = false;
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		client.close();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java
new file mode 100644
index 0000000..5ad4bae
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/StandardNiFiDataPacket.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.nifi;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * An implementation of NiFiDataPacket.
+ */
+public class StandardNiFiDataPacket implements NiFiDataPacket, Serializable {
+	private static final long serialVersionUID = 6364005260220243322L;
+
+	private final byte[] content;
+	private final Map<String, String> attributes;
+
+	public StandardNiFiDataPacket(final byte[] content, final Map<String, String> attributes) {
+		this.content = content;
+		this.attributes = attributes;
+	}
+
+	@Override
+	public byte[] getContent() {
+		return content;
+	}
+
+	@Override
+	public Map<String, String> getAttributes() {
+		return attributes;
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java
new file mode 100644
index 0000000..572f949
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSinkTopologyExample.java
@@ -0,0 +1,55 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.flink.streaming.connectors.nifi.examples;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.connectors.nifi.NiFiDataPacket;
+import org.apache.flink.streaming.connectors.nifi.NiFiDataPacketBuilder;
+import org.apache.flink.streaming.connectors.nifi.NiFiSink;
+import org.apache.flink.streaming.connectors.nifi.StandardNiFiDataPacket;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+import org.apache.nifi.remote.client.SiteToSiteClientConfig;
+
+import java.util.HashMap;
+
+/**
+ * An example topology that sends data to a NiFi input port named "Data from Flink".
+ */
+public class NiFiSinkTopologyExample {
+
+	public static void main(String[] args) throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		SiteToSiteClientConfig clientConfig = new SiteToSiteClient.Builder()
+				.url("http://localhost:8080/nifi")
+				.portName("Data from Flink")
+				.buildConfig();
+
+		DataStreamSink<String> dataStream = env.fromElements("one", "two", "three", "four", "five", "q")
+				.addSink(new NiFiSink<>(clientConfig, new NiFiDataPacketBuilder<String>() {
+					@Override
+					public NiFiDataPacket createNiFiDataPacket(String s, RuntimeContext ctx) {
+						return new StandardNiFiDataPacket(s.getBytes(), new HashMap<String,String>());
+					}
+				}));
+
+		env.execute();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.java b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.java
new file mode 100644
index 0000000..79c9a1c
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/examples/NiFiSourceTopologyExample.java
@@ -0,0 +1,58 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.flink.streaming.connectors.nifi.examples;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.nifi.NiFiDataPacket;
+import org.apache.flink.streaming.connectors.nifi.NiFiSource;
+import org.apache.nifi.remote.client.SiteToSiteClient;
+import org.apache.nifi.remote.client.SiteToSiteClientConfig;
+
+import java.nio.charset.Charset;
+
+/**
+ * An example topology that pulls data from a NiFi output port named "Data for Flink".
+ */
+public class NiFiSourceTopologyExample {
+
+	public static void main(String[] args) throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		SiteToSiteClientConfig clientConfig = new SiteToSiteClient.Builder()
+				.url("http://localhost:8080/nifi")
+				.portName("Data for Flink")
+				.requestBatchCount(5)
+				.buildConfig();
+
+		SourceFunction<NiFiDataPacket> nifiSource = new NiFiSource(clientConfig);
+		DataStream<NiFiDataPacket> streamSource = env.addSource(nifiSource).setParallelism(2);
+
+		DataStream<String> dataStream = streamSource.map(new MapFunction<NiFiDataPacket, String>() {
+			@Override
+			public String map(NiFiDataPacket value) throws Exception {
+				return new String(value.getContent(), Charset.defaultCharset());
+			}
+		});
+
+		dataStream.print();
+		env.execute();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml b/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml
new file mode 100644
index 0000000..d373d63
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml
@@ -0,0 +1,16 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements. See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License. You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<template><description></description><name>NiFi_Flink</name><snippet><connections><id>34acfdda-dd21-48c0-8779-95d0e258f5cb</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>8b8b6a2f-ee24-4f32-a178-248f3a3f5482</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><selectedRelationships>success</selectedRelationships><source><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>769242e5-ee04-4656-a684-ca661a18eed6</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>59574e3b-1ba7-4343-b265-af1b67923a85</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThresh
 old>0</backPressureObjectThreshold><destination><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>48042218-a51e-45c7-bd30-2290bba8b191</id><type>OUTPUT_PORT</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><selectedRelationships>success</selectedRelationships><source><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>8b8b6a2f-ee24-4f32-a178-248f3a3f5482</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>46c9343f-f732-4e2d-98e1-13caab5d2f5e</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>e3a9026f-dae1-42ca-851c-02d9fda22094</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><source><groupI
 d>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>cd8c5227-cfa9-4603-9472-b2234d7bd741</id><type>INPUT_PORT</type></source><zIndex>0</zIndex></connections><inputPorts><id>cd8c5227-cfa9-4603-9472-b2234d7bd741</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>395.0</x><y>520.0</y></position><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><name>Data from Flink</name><state>RUNNING</state><transmitting>false</transmitting><type>INPUT_PORT</type></inputPorts><outputPorts><id>48042218-a51e-45c7-bd30-2290bba8b191</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>1616.0</x><y>259.0</y></position><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><name>Data for Flink</name><state>RUNNING</state><transmitting>false</transmitting><type>OUTPUT_PORT</type></outputPorts><processors><id>769242e5-ee04-4656-a684-ca661a18eed6</id><parentGroupId>0f854f2b-239f-45f0-bfed-48
 b5b23f7928</parentGroupId><position><x>389.0</x><y>231.0</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><defaultConcurrentTasks><entry><key>TIMER_DRIVEN</key><value>1</value></entry><entry><key>EVENT_DRIVEN</key><value>0</value></entry><entry><key>CRON_DRIVEN</key><value>1</value></entry></defaultConcurrentTasks><defaultSchedulingPeriod><entry><key>TIMER_DRIVEN</key><value>0 sec</value></entry><entry><key>CRON_DRIVEN</key><value>* * * * * ?</value></entry></defaultSchedulingPeriod><descriptors><entry><key>File Size</key><value><description>The size of the file that will be used</description><displayName>File Size</displayName><dynamic>false</dynamic><name>File Size</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Batch Size</key><value><defaultValue>1</defaultValue><description>The number of FlowFiles to be transferr
 ed in each invocation</description><displayName>Batch Size</displayName><dynamic>false</dynamic><name>Batch Size</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Data Format</key><value><allowableValues><displayName>Binary</displayName><value>Binary</value></allowableValues><allowableValues><displayName>Text</displayName><value>Text</value></allowableValues><defaultValue>Binary</defaultValue><description>Specifies whether the data should be Text or Binary</description><displayName>Data Format</displayName><dynamic>false</dynamic><name>Data Format</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Unique FlowFiles</key><value><allowableValues><displayName>true</displayName><value>true</value></allowableValues><allowableValues><displayName>false</displayName><value>false</value></allowableValues><defaultValue>false</defaultValue><description>If true, ea
 ch FlowFile that is generated will be unique. If false, a random value will be generated and all FlowFiles will get the same content but this offers much higher throughput</description><displayName>Unique FlowFiles</displayName><dynamic>false</dynamic><name>Unique FlowFiles</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry></descriptors><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>File Size</key><value>1 b</value></entry><entry><key>Batch Size</key><value>1</value></entry><entry><key>Data Format</key><value>Binary</value></entry><entry><key>Unique FlowFiles</key><value>false</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>2 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GenerateFlowFile</name><relationships><autoTerminate>false</autoTerminate><description></des
 cription><name>success</name></relationships><state>STOPPED</state><style/><supportsEventDriven>false</supportsEventDriven><supportsParallelProcessing>true</supportsParallelProcessing><type>org.apache.nifi.processors.standard.GenerateFlowFile</type></processors><processors><id>e3a9026f-dae1-42ca-851c-02d9fda22094</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>826.0</x><y>499.0</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><defaultConcurrentTasks><entry><key>TIMER_DRIVEN</key><value>1</value></entry><entry><key>EVENT_DRIVEN</key><value>0</value></entry><entry><key>CRON_DRIVEN</key><value>1</value></entry></defaultConcurrentTasks><defaultSchedulingPeriod><entry><key>TIMER_DRIVEN</key><value>0 sec</value></entry><entry><key>CRON_DRIVEN</key><value>* * * * * ?</value></entry></defaultSchedulingPeriod><descriptors><entry><key>Log Level</key><value><al
 lowableValues><displayName>trace</displayName><value>trace</value></allowableValues><allowableValues><displayName>debug</displayName><value>debug</value></allowableValues><allowableValues><displayName>info</displayName><value>info</value></allowableValues><allowableValues><displayName>warn</displayName><value>warn</value></allowableValues><allowableValues><displayName>error</displayName><value>error</value></allowableValues><defaultValue>info</defaultValue><description>The Log Level to use when logging the Attributes</description><displayName>Log Level</displayName><dynamic>false</dynamic><name>Log Level</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Log Payload</key><value><allowableValues><displayName>true</displayName><value>true</value></allowableValues><allowableValues><displayName>false</displayName><value>false</value></allowableValues><defaultValue>false</defaultValue><description>If true, the FlowFile's p
 ayload will be logged, in addition to its attributes; otherwise, just the Attributes will be logged.</description><displayName>Log Payload</displayName><dynamic>false</dynamic><name>Log Payload</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Attributes to Log</key><value><description>A comma-separated list of Attributes to Log. If not specified, all attributes will be logged.</description><displayName>Attributes to Log</displayName><dynamic>false</dynamic><name>Attributes to Log</name><required>false</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Attributes to Ignore</key><value><description>A comma-separated list of Attributes to ignore. If not specified, no attributes will be ignored.</description><displayName>Attributes to Ignore</displayName><dynamic>false</dynamic><name>Attributes to Ignore</name><required>false</required><sensitive>false</sensitive><supportsEl>
 false</supportsEl></value></entry><entry><key>Log prefix</key><value><description>Log prefix appended to the log lines. It helps to distinguish the output of multiple LogAttribute processors.</description><displayName>Log prefix</displayName><dynamic>false</dynamic><name>Log prefix</name><required>false</required><sensitive>false</sensitive><supportsEl>true</supportsEl></value></entry></descriptors><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Log Level</key></entry><entry><key>Log Payload</key><value>true</value></entry><entry><key>Attributes to Log</key></entry><entry><key>Attributes to Ignore</key></entry><entry><key>Log prefix</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>LogAttribute</name><relationships><autoTerminate>true</autoTerminate><description>All FlowFil
 es are routed to this relationship</description><name>success</name></relationships><state>RUNNING</state><style/><supportsEventDriven>true</supportsEventDriven><supportsParallelProcessing>true</supportsParallelProcessing><type>org.apache.nifi.processors.standard.LogAttribute</type></processors><processors><id>8b8b6a2f-ee24-4f32-a178-248f3a3f5482</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>1000.0</x><y>231.0</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><defaultConcurrentTasks><entry><key>TIMER_DRIVEN</key><value>1</value></entry><entry><key>EVENT_DRIVEN</key><value>0</value></entry><entry><key>CRON_DRIVEN</key><value>1</value></entry></defaultConcurrentTasks><defaultSchedulingPeriod><entry><key>TIMER_DRIVEN</key><value>0 sec</value></entry><entry><key>CRON_DRIVEN</key><value>* * * * * ?</value></entry></defaultSchedulingPeriod><descriptors><e
 ntry><key>Regular Expression</key><value><defaultValue>(?s:^.*$)</defaultValue><description>The Regular Expression to search for in the FlowFile content</description><displayName>Regular Expression</displayName><dynamic>false</dynamic><name>Regular Expression</name><required>true</required><sensitive>false</sensitive><supportsEl>true</supportsEl></value></entry><entry><key>Replacement Value</key><value><defaultValue>$1</defaultValue><description>The value to replace the regular expression with. Back-references to Regular Expression capturing groups are supported, but back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value.</description><displayName>Replacement Value</displayName><dynamic>false</dynamic><name>Replacement Value</name><required>true</required><sensitive>false</sensitive><supportsEl>true</supportsEl></value></entry><entry><key>Character Set</key><value><defaultValue>UTF-8</defaultValue><description>The
  Character Set in which the file is encoded</description><displayName>Character Set</displayName><dynamic>false</dynamic><name>Character Set</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Maximum Buffer Size</key><value><defaultValue>1 MB</defaultValue><description>Specifies the maximum amount of data to buffer (per file or per line, depending on the Evaluation Mode) in order to apply the regular expressions. If 'Entire Text' (in Evaluation Mode) is selected and the FlowFile is larger than this value, the FlowFile will be routed to 'failure'. In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. This value is ignored and the buffer is not used if 'Regular Expression' is set to '.*'</description><displayName>Maximum Buffer 
 Size</displayName><dynamic>false</dynamic><name>Maximum Buffer Size</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Evaluation Mode</key><value><allowableValues><displayName>Line-by-Line</displayName><value>Line-by-Line</value></allowableValues><allowableValues><displayName>Entire text</displayName><value>Entire text</value></allowableValues><defaultValue>Entire text</defaultValue><description>Evaluate the 'Regular Expression' against each line (Line-by-Line) or buffer the entire file into memory (Entire Text) and then evaluate the 'Regular Expression'.</description><displayName>Evaluation Mode</displayName><dynamic>false</dynamic><name>Evaluation Mode</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry></descriptors><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Regular Expression</key><value>(?s:^.*$)</value><
 /entry><entry><key>Replacement Value</key><value>blah blah</value></entry><entry><key>Character Set</key><value>UTF-8</value></entry><entry><key>Maximum Buffer Size</key><value>1 MB</value></entry><entry><key>Evaluation Mode</key><value>Entire text</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>ReplaceText</name><relationships><autoTerminate>true</autoTerminate><description>FlowFiles that could not be updated are routed to this relationship</description><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><description>FlowFiles that have been successfully updated are routed to this relationship, as well as FlowFiles whose content does not match the given Regular Expression</description><name>success</name></relationships><state>RUNNING</state><style/><supportsEventDriven>true</supports
 EventDriven><supportsParallelProcessing>true</supportsParallelProcessing><type>org.apache.nifi.processors.standard.ReplaceText</type></processors></snippet><timestamp>09/30/2015 09:10:38 EDT</timestamp></template>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml b/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
new file mode 100644
index 0000000..314289a
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
@@ -0,0 +1,59 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-streaming-connectors-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-rabbitmq</artifactId>
+	<name>flink-connector-rabbitmq</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<rabbitmq.version>3.3.1</rabbitmq.version>
+	</properties>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.rabbitmq</groupId>
+			<artifactId>amqp-client</artifactId>
+			<version>${rabbitmq.version}</version>
+		</dependency>
+
+	</dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
new file mode 100644
index 0000000..fa729d6
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq;
+
+import java.io.IOException;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+
+public class RMQSink<IN> extends RichSinkFunction<IN> {
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(RMQSink.class);
+
+	private String QUEUE_NAME;
+	private String HOST_NAME;
+	private transient ConnectionFactory factory;
+	private transient Connection connection;
+	private transient Channel channel;
+	private SerializationSchema<IN, byte[]> schema;
+
+	public RMQSink(String HOST_NAME, String QUEUE_NAME, SerializationSchema<IN, byte[]> schema) {
+		this.HOST_NAME = HOST_NAME;
+		this.QUEUE_NAME = QUEUE_NAME;
+		this.schema = schema;
+	}
+
+	/**
+	 * Initializes the connection to RMQ.
+	 */
+	public void initializeConnection() {
+		factory = new ConnectionFactory();
+		factory.setHost(HOST_NAME);
+		try {
+			connection = factory.newConnection();
+			channel = connection.createChannel();
+			channel.queueDeclare(QUEUE_NAME, false, false, false, null);
+
+		} catch (IOException e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	/**
+	 * Called when new data arrives to the sink, and forwards it to RMQ.
+	 * 
+	 * @param value
+	 *            The incoming data
+	 */
+	@Override
+	public void invoke(IN value) {
+		try {
+			byte[] msg = schema.serialize(value);
+
+			channel.basicPublish("", QUEUE_NAME, null, msg);
+
+		} catch (IOException e) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Cannot send RMQ message {} at {}", QUEUE_NAME, HOST_NAME);
+			}
+		}
+
+	}
+
+	/**
+	 * Closes the connection.
+	 */
+	private void closeChannel() {
+		try {
+			channel.close();
+			connection.close();
+		} catch (IOException e) {
+			throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME
+					+ " at " + HOST_NAME, e);
+		}
+
+	}
+
+	@Override
+	public void open(Configuration config) {
+		initializeConnection();
+	}
+
+	@Override
+	public void close() {
+		closeChannel();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
new file mode 100644
index 0000000..50149dc
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq;
+
+import java.io.IOException;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.ConnectorSource;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.QueueingConsumer;
+
+public class RMQSource<OUT> extends ConnectorSource<OUT> {
+	private static final long serialVersionUID = 1L;
+
+	private final String QUEUE_NAME;
+	private final String HOST_NAME;
+
+	private transient ConnectionFactory factory;
+	private transient Connection connection;
+	private transient Channel channel;
+	private transient QueueingConsumer consumer;
+	private transient QueueingConsumer.Delivery delivery;
+
+	private transient volatile boolean running;
+
+	public RMQSource(String HOST_NAME, String QUEUE_NAME,
+			DeserializationSchema<OUT> deserializationSchema) {
+		super(deserializationSchema);
+		this.HOST_NAME = HOST_NAME;
+		this.QUEUE_NAME = QUEUE_NAME;
+	}
+
+	/**
+	 * Initializes the connection to RMQ.
+	 */
+	private void initializeConnection() {
+		factory = new ConnectionFactory();
+		factory.setHost(HOST_NAME);
+		try {
+			connection = factory.newConnection();
+			channel = connection.createChannel();
+			channel.queueDeclare(QUEUE_NAME, false, false, false, null);
+			consumer = new QueueingConsumer(channel);
+			channel.basicConsume(QUEUE_NAME, true, consumer);
+		} catch (IOException e) {
+			throw new RuntimeException("Cannot create RMQ connection with " + QUEUE_NAME + " at "
+					+ HOST_NAME, e);
+		}
+	}
+
+	@Override
+	public void open(Configuration config) throws Exception {
+		initializeConnection();
+		running = true;
+	}
+
+	@Override
+	public void close() throws Exception {
+		super.close();
+		try {
+			connection.close();
+		} catch (IOException e) {
+			throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME
+					+ " at " + HOST_NAME, e);
+		}
+	}
+
+	@Override
+	public void run(SourceContext<OUT> ctx) throws Exception {
+		while (running) {
+			delivery = consumer.nextDelivery();
+
+			OUT result = schema.deserialize(delivery.getBody());
+			if (schema.isEndOfStream(result)) {
+				break;
+			}
+
+			ctx.collect(result);
+		}
+	}
+
+	@Override
+	public void cancel() {
+		running = false;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
new file mode 100644
index 0000000..1f85862
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.rabbitmq;
+
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+public class RMQTopology {
+
+	public static void main(String[] args) throws Exception {
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+		@SuppressWarnings("unused")
+		DataStreamSink<String> dataStream1 = env.addSource(
+				new RMQSource<String>("localhost", "hello", new SimpleStringSchema())).print();
+
+		@SuppressWarnings("unused")
+		DataStreamSink<String> dataStream2 = env.fromElements("one", "two", "three", "four", "five",
+				"q").addSink(
+				new RMQSink<String>("localhost", "hello", new StringToByteSerializer()));
+
+		env.execute();
+	}
+
+	public static class StringToByteSerializer implements SerializationSchema<String, byte[]> {
+
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public byte[] serialize(String element) {
+			return element.getBytes();
+		}
+	}
+}


[35/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
deleted file mode 100644
index fbecbd1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-
-/**
- * {@link StreamOperator} for streaming sources.
- */
-public class StreamSource<T> extends AbstractUdfStreamOperator<T, SourceFunction<T>> implements StreamOperator<T> {
-
-	private static final long serialVersionUID = 1L;
-	private transient SourceFunction.SourceContext<T> ctx;
-
-	public StreamSource(SourceFunction<T> sourceFunction) {
-		super(sourceFunction);
-
-		this.chainingStrategy = ChainingStrategy.HEAD;
-	}
-
-	public void run(final Object lockingObject, final Output<StreamRecord<T>> collector) throws Exception {
-		final ExecutionConfig executionConfig = getExecutionConfig();
-		
-		if (userFunction instanceof EventTimeSourceFunction) {
-			ctx = new ManualWatermarkContext<T>(lockingObject, collector);
-		} else if (executionConfig.getAutoWatermarkInterval() > 0) {
-			ctx = new AutomaticWatermarkContext<T>(lockingObject, collector, executionConfig);
-		} else if (executionConfig.areTimestampsEnabled()) {
-			ctx = new NonWatermarkContext<T>(lockingObject, collector);
-		} else {
-			ctx = new NonTimestampContext<T>(lockingObject, collector);
-		}
-
-		userFunction.run(ctx);
-
-		// This will mostly emit a final +Inf Watermark to make the Watermark logic work
-		// when some sources finish before others do
-		ctx.close();
-	}
-
-	public void cancel() {
-		userFunction.cancel();
-		ctx.close();
-	}
-
-	/**
-	 * {@link SourceFunction.SourceContext} to be used for sources that don't emit watermarks.
-	 * In addition to {@link NonWatermarkContext} this will also not attach timestamps to sources.
-	 * (Technically it will always set the timestamp to 0).
-	 */
-	public static class NonTimestampContext<T> implements SourceFunction.SourceContext<T> {
-
-		private final Object lockingObject;
-		private final Output<StreamRecord<T>> output;
-		private final StreamRecord<T> reuse;
-
-		public NonTimestampContext(Object lockingObjectParam, Output<StreamRecord<T>> outputParam) {
-			this.lockingObject = lockingObjectParam;
-			this.output = outputParam;
-			this.reuse = new StreamRecord<T>(null);
-		}
-
-		@Override
-		public void collect(T element) {
-			output.collect(reuse.replace(element, 0));
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" +
-					" elements with a timestamp. See interface ManualTimestampSourceFunction" +
-					" if you want to manually assign timestamps to elements.");
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" +
-					" elements with a timestamp. See interface ManualTimestampSourceFunction" +
-					" if you want to manually assign timestamps to elements.");
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lockingObject;
-		}
-
-		@Override
-		public void close() {}
-	}
-
-	/**
-	 * {@link SourceFunction.SourceContext} to be used for sources that don't emit watermarks.
-	 */
-	public static class NonWatermarkContext<T> implements SourceFunction.SourceContext<T> {
-
-		private final Object lockingObject;
-		private final Output<StreamRecord<T>> output;
-		private final StreamRecord<T> reuse;
-
-		public NonWatermarkContext(Object lockingObjectParam, Output<StreamRecord<T>> outputParam) {
-			this.lockingObject = lockingObjectParam;
-			this.output = outputParam;
-			this.reuse = new StreamRecord<T>(null);
-		}
-
-		@Override
-		public void collect(T element) {
-			long currentTime = System.currentTimeMillis();
-			output.collect(reuse.replace(element, currentTime));
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" +
-					" elements with a timestamp. See interface ManualTimestampSourceFunction" +
-					" if you want to manually assign timestamps to elements.");
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" +
-					" elements with a timestamp. See interface ManualTimestampSourceFunction" +
-					" if you want to manually assign timestamps to elements.");
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lockingObject;
-		}
-
-		@Override
-		public void close() {}
-	}
-
-	/**
-	 * {@link SourceFunction.SourceContext} to be used for sources with automatic timestamps
-	 * and watermark emission.
-	 */
-	public static class AutomaticWatermarkContext<T> implements SourceFunction.SourceContext<T> {
-
-		private final ScheduledExecutorService scheduleExecutor;
-		private final ScheduledFuture<?> watermarkTimer;
-		private final long watermarkInterval;
-
-		private final Object lockingObject;
-		private final Output<StreamRecord<T>> output;
-		private final StreamRecord<T> reuse;
-
-		private volatile long lastWatermarkTime;
-
-		public AutomaticWatermarkContext(Object lockingObjectParam,
-				Output<StreamRecord<T>> outputParam,
-				ExecutionConfig executionConfig) {
-			this.lockingObject = lockingObjectParam;
-			this.output = outputParam;
-			this.reuse = new StreamRecord<T>(null);
-
-			watermarkInterval = executionConfig.getAutoWatermarkInterval();
-
-			scheduleExecutor = Executors.newScheduledThreadPool(1);
-
-			watermarkTimer = scheduleExecutor.scheduleAtFixedRate(new Runnable() {
-				@Override
-				public void run() {
-					long currentTime = System.currentTimeMillis();
-					// align the watermarks across all machines. this will ensure that we
-					// don't have watermarks that creep along at different intervals because
-					// the machine clocks are out of sync
-					long watermarkTime = currentTime - (currentTime % watermarkInterval);
-					if (currentTime > watermarkTime && watermarkTime - lastWatermarkTime >= watermarkInterval) {
-						synchronized (lockingObject) {
-							if (currentTime > watermarkTime && watermarkTime - lastWatermarkTime >= watermarkInterval) {
-								output.emitWatermark(new Watermark(watermarkTime));
-								lastWatermarkTime = watermarkTime;
-							}
-						}
-					}
-				}
-			}, 0, watermarkInterval, TimeUnit.MILLISECONDS);
-
-		}
-
-		@Override
-		public void collect(T element) {
-			synchronized (lockingObject) {
-				long currentTime = System.currentTimeMillis();
-				output.collect(reuse.replace(element, currentTime));
-
-				long watermarkTime = currentTime - (currentTime % watermarkInterval);
-				if (currentTime > watermarkTime && watermarkTime - lastWatermarkTime >= watermarkInterval) {
-					output.emitWatermark(new Watermark(watermarkTime));
-					lastWatermarkTime = watermarkTime;
-				}
-			}
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" +
-					" elements with a timestamp. See interface ManualTimestampSourceFunction" +
-					" if you want to manually assign timestamps to elements.");
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			throw new UnsupportedOperationException("Automatic-Timestamp sources cannot emit" +
-					" elements with a timestamp. See interface ManualTimestampSourceFunction" +
-					" if you want to manually assign timestamps to elements.");
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lockingObject;
-		}
-
-		@Override
-		public void close() {
-			watermarkTimer.cancel(true);
-			scheduleExecutor.shutdownNow();
-			// emit one last +Inf watermark to make downstream watermark processing work
-			// when some sources close early
-			synchronized (lockingObject) {
-				output.emitWatermark(new Watermark(Long.MAX_VALUE));
-			}
-		}
-	}
-
-	/**
-	 * {@link SourceFunction.SourceContext} to be used for sources with manual timestamp
-	 * assignment and manual watermark emission.
-	 */
-	public static class ManualWatermarkContext<T> implements SourceFunction.SourceContext<T> {
-
-		private final Object lockingObject;
-		private final Output<StreamRecord<T>> output;
-		private final StreamRecord<T> reuse;
-
-		public ManualWatermarkContext(Object lockingObject, Output<StreamRecord<T>> output) {
-			this.lockingObject = lockingObject;
-			this.output = output;
-			this.reuse = new StreamRecord<T>(null);
-		}
-
-		@Override
-		public void collect(T element) {
-			throw new UnsupportedOperationException("Manual-Timestamp sources can only emit" +
-					" elements with a timestamp. Please use collectWithTimestamp().");
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			synchronized (lockingObject) {
-				output.collect(reuse.replace(element, timestamp));
-			}
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			output.emitWatermark(mark);
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lockingObject;
-		}
-
-		@Override
-		public void close() {
-			// emit one last +Inf watermark to make downstream watermark processing work
-			// when some sources close early
-			synchronized (lockingObject) {
-				output.emitWatermark(new Watermark(Long.MAX_VALUE));
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
deleted file mode 100644
index 87a9abd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
-import org.apache.flink.api.common.functions.util.AbstractRuntimeUDFContext;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Implementation of the {@link org.apache.flink.api.common.functions.RuntimeContext},
- * for streaming operators.
- */
-public class StreamingRuntimeContext extends AbstractRuntimeUDFContext {
-
-	/** The operator to which this function belongs */
-	private final AbstractStreamOperator<?> operator;
-	
-	/** The task environment running the operator */
-	private final Environment taskEnvironment;
-	
-	/** The key/value state, if the user-function requests it */
-	private HashMap<String, OperatorState<?>> keyValueStates;
-	
-	/** Type of the values stored in the state, to make sure repeated requests of the state are consistent */
-	private HashMap<String, TypeInformation<?>> stateTypeInfos;
-	
-	
-	public StreamingRuntimeContext(AbstractStreamOperator<?> operator,
-									Environment env, Map<String, Accumulator<?, ?>> accumulators) {
-		super(env.getTaskName(),
-				env.getNumberOfSubtasks(),
-				env.getIndexInSubtaskGroup(),
-				env.getUserClassLoader(),
-				operator.getExecutionConfig(),
-				accumulators,
-				env.getDistributedCacheEntries());
-		
-		this.operator = operator;
-		this.taskEnvironment = env;
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Returns the input split provider associated with the operator.
-	 * 
-	 * @return The input split provider.
-	 */
-	public InputSplitProvider getInputSplitProvider() {
-		return taskEnvironment.getInputSplitProvider();
-	}
-
-	/**
-	 * Register a timer callback. At the specified time the {@link Triggerable } will be invoked.
-	 * This call is guaranteed to not happen concurrently with method calls on the operator.
-	 *
-	 * @param time The absolute time in milliseconds.
-	 * @param target The target to be triggered.
-	 */
-	public void registerTimer(long time, Triggerable target) {
-		operator.registerTimer(time, target);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  broadcast variables
-	// ------------------------------------------------------------------------
-
-	@Override
-	public <RT> List<RT> getBroadcastVariable(String name) {
-		throw new UnsupportedOperationException("Broadcast variables can only be used in DataSet programs");
-	}
-
-	@Override
-	public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
-		throw new UnsupportedOperationException("Broadcast variables can only be used in DataSet programs");
-	}
-
-	// ------------------------------------------------------------------------
-	//  key/value state
-	// ------------------------------------------------------------------------
-
-	@Override
-	public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
-		requireNonNull(stateType, "The state type class must not be null");
-
-		TypeInformation<S> typeInfo;
-		try {
-			typeInfo = TypeExtractor.getForClass(stateType);
-		}
-		catch (Exception e) {
-			throw new RuntimeException("Cannot analyze type '" + stateType.getName() + 
-					"' from the class alone, due to generic type parameters. " +
-					"Please specify the TypeInformation directly.", e);
-		}
-		
-		return getKeyValueState(name, typeInfo, defaultState);
-	}
-
-	@Override
-	public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
-		requireNonNull(name, "The name of the state must not be null");
-		requireNonNull(stateType, "The state type information must not be null");
-		
-		OperatorState<?> previousState;
-		
-		// check if this is a repeated call to access the state 
-		if (this.stateTypeInfos != null && this.keyValueStates != null &&
-				(previousState = this.keyValueStates.get(name)) != null) {
-			
-			// repeated call
-			TypeInformation<?> previousType;
-			if (stateType.equals((previousType = this.stateTypeInfos.get(name)))) {
-				// valid case, same type requested again
-				@SuppressWarnings("unchecked")
-				OperatorState<S> previous = (OperatorState<S>) previousState;
-				return previous;
-			}
-			else {
-				// invalid case, different type requested this time
-				throw new IllegalStateException("Cannot initialize key/value state for type " + stateType +
-						" ; The key/value state has already been created and initialized for a different type: " +
-						previousType);
-			}
-		}
-		else {
-			// first time access to the key/value state
-			if (this.stateTypeInfos == null) {
-				this.stateTypeInfos = new HashMap<>();
-			}
-			if (this.keyValueStates == null) {
-				this.keyValueStates = new HashMap<>();
-			}
-			
-			try {
-				OperatorState<S> state = operator.createKeyValueState(name, stateType, defaultState);
-				this.keyValueStates.put(name, state);
-				this.stateTypeInfos.put(name, stateType);
-				return state;
-			}
-			catch (RuntimeException e) {
-				throw e;
-			}
-			catch (Exception e) {
-				throw new RuntimeException("Cannot initialize the key/value state", e);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java
deleted file mode 100644
index 62514fc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TimestampedCollector.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-/**
- * Wrapper around an {@link Output} for user functions that expect a {@link Collector}.
- * Before giving the {@link TimestampedCollector} to a user function you must set
- * the {@link Instant timestamp} that should be attached to emitted elements. Most operators
- * would set the {@link Instant timestamp} of the incoming {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord} here.
- *
- * @param <T> The type of the elments that can be emitted.
- */
-public class TimestampedCollector<T> implements Collector<T> {
-	
-	private final Output<StreamRecord<T>> output;
-
-	private final StreamRecord<T> reuse;
-	
-	private long timestamp;
-	
-
-	/**
-	 * Creates a new {@link TimestampedCollector} that wraps the given {@link Output}.
-	 */
-	public TimestampedCollector(Output<StreamRecord<T>> output) {
-		this.output = output;
-		this.reuse = new StreamRecord<T>(null);
-	}
-
-	@Override
-	public void collect(T record) {
-		output.collect(reuse.replace(record, timestamp));
-	}
-
-	/**
-	 * Sets the {@link Instant timestamp} that is attached to elements that get emitted using
-	 * {@link #collect}
-	 * @param timestamp The timestamp in milliseconds
-	 */
-	public void setTimestamp(long timestamp) {
-		this.timestamp = timestamp;
-	}
-
-	@Override
-	public void close() {
-		output.close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java
deleted file mode 100644
index afc6d1b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/TwoInputStreamOperator.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators;
-
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Interface for stream operators with two inputs. Use
- * {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator} as a base class if
- * you want to implement a custom operator.
- * 
- * @param <IN1> The input type of the operator
- * @param <IN2> The input type of the operator
- * @param <OUT> The output type of the operator
- */
-public interface TwoInputStreamOperator<IN1, IN2, OUT> extends StreamOperator<OUT> {
-
-	/**
-	 * Processes one element that arrived on the first input of this two-input operator.
-	 * This method is guaranteed to not be called concurrently with other methods of the operator.
-	 */
-	public void processElement1(StreamRecord<IN1> element) throws Exception;
-
-	/**
-	 * Processes one element that arrived on the second input of this two-input operator.
-	 * This method is guaranteed to not be called concurrently with other methods of the operator.
-	 */
-	public void processElement2(StreamRecord<IN2> element) throws Exception;
-
-	/**
-	 * Processes a {@link Watermark} that arrived on the first input of this two-input operator.
-	 * This method is guaranteed to not be called concurrently with other methods of the operator.
-	 *
-	 * @see org.apache.flink.streaming.api.watermark.Watermark
-	 */
-	public void processWatermark1(Watermark mark) throws Exception;
-
-	/**
-	 * Processes a {@link Watermark} that arrived on the second input of this two-input operator.
-	 * This method is guaranteed to not be called concurrently with other methods of the operator.
-	 *
-	 * @see org.apache.flink.streaming.api.watermark.Watermark
-	 */
-	public void processWatermark2(Watermark mark) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java
deleted file mode 100644
index 806cef2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.co;
-
-import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class CoStreamFlatMap<IN1, IN2, OUT>
-		extends AbstractUdfStreamOperator<OUT, CoFlatMapFunction<IN1, IN2, OUT>>
-		implements TwoInputStreamOperator<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	private transient TimestampedCollector<OUT> collector;
-
-	// We keep track of watermarks from both inputs, the combined input is the minimum
-	// Once the minimum advances we emit a new watermark for downstream operators
-	private long combinedWatermark = Long.MIN_VALUE;
-	private long input1Watermark = Long.MIN_VALUE;
-	private long input2Watermark = Long.MIN_VALUE;
-
-	public CoStreamFlatMap(CoFlatMapFunction<IN1, IN2, OUT> flatMapper) {
-		super(flatMapper);
-	}
-
-	@Override
-	public void open() throws Exception {
-		super.open();
-		collector = new TimestampedCollector<OUT>(output);
-	}
-
-	@Override
-	public void processElement1(StreamRecord<IN1> element) throws Exception {
-		collector.setTimestamp(element.getTimestamp());
-		userFunction.flatMap1(element.getValue(), collector);
-
-	}
-
-	@Override
-	public void processElement2(StreamRecord<IN2> element) throws Exception {
-		collector.setTimestamp(element.getTimestamp());
-		userFunction.flatMap2(element.getValue(), collector);
-	}
-
-	@Override
-	public void processWatermark1(Watermark mark) throws Exception {
-		input1Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-
-	@Override
-	public void processWatermark2(Watermark mark) throws Exception {
-		input2Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-
-	protected TimestampedCollector<OUT> getCollector() {
-		return collector;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java
deleted file mode 100644
index e34e727..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamMap.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.operators.co;
-
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-public class CoStreamMap<IN1, IN2, OUT>
-		extends AbstractUdfStreamOperator<OUT, CoMapFunction<IN1, IN2, OUT>>
-		implements TwoInputStreamOperator<IN1, IN2, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	// We keep track of watermarks from both inputs, the combined input is the minimum
-	// Once the minimum advances we emit a new watermark for downstream operators
-	private long combinedWatermark = Long.MIN_VALUE;
-	private long input1Watermark = Long.MIN_VALUE;
-	private long input2Watermark = Long.MIN_VALUE;
-
-	public CoStreamMap(CoMapFunction<IN1, IN2, OUT> mapper) {
-		super(mapper);
-	}
-
-	@Override
-	public void processElement1(StreamRecord<IN1> element) throws Exception {
-		output.collect(element.replace(userFunction.map1(element.getValue())));
-	}
-
-	@Override
-	public void processElement2(StreamRecord<IN2> element) throws Exception {
-		output.collect(element.replace(userFunction.map2(element.getValue())));
-	}
-
-	@Override
-	public void processWatermark1(Watermark mark) throws Exception {
-		input1Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-
-	@Override
-	public void processWatermark2(Watermark mark) throws Exception {
-		input2Watermark = mark.getTimestamp();
-		long newMin = Math.min(input1Watermark, input2Watermark);
-		if (newMin > combinedWatermark) {
-			combinedWatermark = newMin;
-			output.emitWatermark(new Watermark(combinedWatermark));
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
deleted file mode 100644
index f9c95f5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * This represents a feedback point in a topology. The type of the feedback elements must not match
- * the type of the upstream {@code StreamTransformation} because the only allowed operations
- * after a {@code CoFeedbackTransformation} are
- * {@link org.apache.flink.streaming.api.transformations.TwoInputTransformation TwoInputTransformations}.
- * The upstream {@code StreamTransformation} will be connected to the first input of the Co-Transform
- * while the feedback edges will be connected to the second input.
- *
- * <p>
- * Both the partitioning of the input and the feedback edges is preserved. They can also have
- * differing partitioning strategies. This requires, however, that the parallelism of the feedback
- * {@code StreamTransformations} must match the parallelism of the input
- * {@code StreamTransformation}.
- *
- * <p>
- * The upstream {@code StreamTransformation} is not wired to this {@code CoFeedbackTransformation}.
- * It is instead directly wired to the {@code TwoInputTransformation} after this
- * {@code CoFeedbackTransformation}.
- *
- * <p>
- * This is different from Iterations in batch processing.
- * @see org.apache.flink.streaming.api.transformations.FeedbackTransformation
- *
- * @param <F> The type of the feedback elements.
- *
- */
-public class CoFeedbackTransformation<F> extends StreamTransformation<F> {
-
-	private final List<StreamTransformation<F>> feedbackEdges;
-
-	private final Long waitTime;
-
-	/**
-	 * Creates a new {@code CoFeedbackTransformation} from the given input.
-	 *
-	 * @param parallelism The parallelism of the upstream {@code StreamTransformatino} and the
-	 *                    feedback edges.
-	 * @param feedbackType The type of the feedback edges
-	 * @param waitTime The wait time of the feedback operator. After the time expires
-	 *                          the operation will close and not receive any more feedback elements.
-	 */
-	public CoFeedbackTransformation(int parallelism,
-			TypeInformation<F> feedbackType,
-			Long waitTime) {
-		super("CoFeedback", feedbackType, parallelism);
-		this.waitTime = waitTime;
-		this.feedbackEdges = Lists.newArrayList();
-	}
-
-	/**
-	 * Adds a feedback edge. The parallelism of the {@code StreamTransformation} must match
-	 * the parallelism of the input {@code StreamTransformation} of the upstream
-	 * {@code StreamTransformation}.
-	 *
-	 * @param transform The new feedback {@code StreamTransformation}.
-	 */
-	public void addFeedbackEdge(StreamTransformation<F> transform) {
-
-		if (transform.getParallelism() != this.getParallelism()) {
-			throw new UnsupportedOperationException(
-					"Parallelism of the feedback stream must match the parallelism of the original" +
-							" stream. Parallelism of original stream: " + this.getParallelism() +
-							"; parallelism of feedback stream: " + transform.getParallelism());
-		}
-
-		feedbackEdges.add(transform);
-	}
-
-	/**
-	 * Returns the list of feedback {@code StreamTransformations}.
-	 */
-	public List<StreamTransformation<F>> getFeedbackEdges() {
-		return feedbackEdges;
-	}
-
-	/**
-	 * Returns the wait time. This is the amount of time that the feedback operator keeps listening
-	 * for feedback elements. Once the time expires the operation will close and will not receive
-	 * further elements.
-	 */
-	public Long getWaitTime() {
-		return waitTime;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		return Collections.<StreamTransformation<?>>singleton(this);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
deleted file mode 100644
index 87c7f16..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This represents a feedback point in a topology.
- *
- * <p>
- * This is different from how iterations work in batch processing. Once a feedback point is defined
- * you can connect one or several {@code StreamTransformations} as a feedback edges. Operations
- * downstream from the feedback point will receive elements from the input of this feedback point
- * and from the feedback edges.
- *
- * <p>
- * Both the partitioning of the input and the feedback edges is preserved. They can also have
- * differing partitioning strategies. This requires, however, that the parallelism of the feedback
- * {@code StreamTransformations} must match the parallelism of the input
- * {@code StreamTransformation}.
- *
- * <p>
- * The type of the input {@code StreamTransformation} and the feedback {@code StreamTransformation}
- * must match.
- *
- * @param <T> The type of the input elements and the feedback elements.
- */
-public class FeedbackTransformation<T> extends StreamTransformation<T> {
-
-	private final StreamTransformation<T> input;
-
-	private final List<StreamTransformation<T>> feedbackEdges;
-
-	private final Long waitTime;
-
-	/**
-	 * Creates a new {@code FeedbackTransformation} from the given input.
-	 *
-	 * @param input The input {@code StreamTransformation}
-	 * @param waitTime The wait time of the feedback operator. After the time expires
-	 *                          the operation will close and not receive any more feedback elements.
-	 */
-	public FeedbackTransformation(StreamTransformation<T> input, Long waitTime) {
-		super("Feedback", input.getOutputType(), input.getParallelism());
-		this.input = input;
-		this.waitTime = waitTime;
-		this.feedbackEdges = Lists.newArrayList();
-	}
-
-	/**
-	 * Returns the input {@code StreamTransformation} of this {@code FeedbackTransformation}.
-	 */
-	public StreamTransformation<T> getInput() {
-		return input;
-	}
-
-	/**
-	 * Adds a feedback edge. The parallelism of the {@code StreamTransformation} must match
-	 * the parallelism of the input {@code StreamTransformation} of this
-	 * {@code FeedbackTransformation}
-	 *
-	 * @param transform The new feedback {@code StreamTransformation}.
-	 */
-	public void addFeedbackEdge(StreamTransformation<T> transform) {
-
-		if (transform.getParallelism() != this.getParallelism()) {
-			throw new UnsupportedOperationException(
-					"Parallelism of the feedback stream must match the parallelism of the original" +
-							" stream. Parallelism of original stream: " + this.getParallelism() +
-							"; parallelism of feedback stream: " + transform.getParallelism());
-		}
-
-		feedbackEdges.add(transform);
-	}
-
-	/**
-	 * Returns the list of feedback {@code StreamTransformations}.
-	 */
-	public List<StreamTransformation<T>> getFeedbackEdges() {
-		return feedbackEdges;
-	}
-
-	/**
-	 * Returns the wait time. This is the amount of time that the feedback operator keeps listening
-	 * for feedback elements. Once the time expires the operation will close and will not receive
-	 * further elements.
-	 */
-	public Long getWaitTime() {
-		return waitTime;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		result.addAll(input.getTransitivePredecessors());
-		return result;
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
deleted file mode 100644
index 031c481..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
+++ /dev/null
@@ -1,127 +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.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This Transformation represents the application of a
- * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator} to one input
- * {@link org.apache.flink.streaming.api.transformations.StreamTransformation}.
- *
- * @param <IN> The type of the elements in the nput {@code StreamTransformation}
- * @param <OUT> The type of the elements that result from this {@code OneInputTransformation}
- */
-public class OneInputTransformation<IN, OUT> extends StreamTransformation<OUT> {
-
-	private final StreamTransformation<IN> input;
-
-	private final OneInputStreamOperator<IN, OUT> operator;
-
-	private KeySelector<IN, ?> stateKeySelector;
-	
-	private TypeInformation<?> stateKeyType;
-
-	/**
-	 * Creates a new {@code OneInputTransformation} from the given input and operator.
-	 *
-	 * @param input The input {@code StreamTransformation}
-	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
-	 * @param operator The {@code TwoInputStreamOperator}
-	 * @param outputType The type of the elements produced by this {@code OneInputTransformation}
-	 * @param parallelism The parallelism of this {@code OneInputTransformation}
-	 */
-	public OneInputTransformation(
-			StreamTransformation<IN> input,
-			String name,
-			OneInputStreamOperator<IN, OUT> operator,
-			TypeInformation<OUT> outputType,
-			int parallelism) {
-		super(name, outputType, parallelism);
-		this.input = input;
-		this.operator = operator;
-	}
-
-	/**
-	 * Returns the input {@code StreamTransformation} of this {@code OneInputTransformation}.
-	 */
-	public StreamTransformation<IN> getInput() {
-		return input;
-	}
-
-	/**
-	 * Returns the {@code TypeInformation} for the elements of the input.
-	 */
-	public TypeInformation<IN> getInputType() {
-		return input.getOutputType();
-	}
-
-	/**
-	 * Returns the {@code TwoInputStreamOperator} of this Transformation.
-	 */
-	public OneInputStreamOperator<IN, OUT> getOperator() {
-		return operator;
-	}
-
-	/**
-	 * Sets the {@link KeySelector} that must be used for partitioning keyed state of this operation.
-	 *
-	 * @param stateKeySelector The {@code KeySelector} to set
-	 */
-	public void setStateKeySelector(KeySelector<IN, ?> stateKeySelector) {
-		this.stateKeySelector = stateKeySelector;
-	}
-
-	/**
-	 * Returns the {@code KeySelector} that must be used for partitioning keyed state in this
-	 * Operation.
-	 *
-	 * @see #setStateKeySelector
-	 */
-	public KeySelector<IN, ?> getStateKeySelector() {
-		return stateKeySelector;
-	}
-
-	public void setStateKeyType(TypeInformation<?> stateKeyType) {
-		this.stateKeyType = stateKeyType;
-	}
-
-	public TypeInformation<?> getStateKeyType() {
-		return stateKeyType;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		result.addAll(input.getTransitivePredecessors());
-		return result;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		operator.setChainingStrategy(strategy);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
deleted file mode 100644
index fa85349..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
+++ /dev/null
@@ -1,82 +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.transformations;
-
-import com.google.common.collect.Lists;
-
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This transformation represents a change of partitioning of the input elements.
- *
- * <p>
- * This does not create a physical operation, it only affects how upstream operations are
- * connected to downstream operations.
- *
- * @param <T> The type of the elements that result from this {@code PartitionTransformation}
- */
-public class PartitionTransformation<T> extends StreamTransformation<T> {
-	
-	private final StreamTransformation<T> input;
-	private final StreamPartitioner<T> partitioner;
-
-	/**
-	 * Creates a new {@code PartitionTransformation} from the given input and
-	 * {@link StreamPartitioner}.
-	 *
-	 * @param input The input {@code StreamTransformation}
-	 * @param partitioner The {@code StreamPartitioner}
-	 */
-	public PartitionTransformation(StreamTransformation<T> input, StreamPartitioner<T> partitioner) {
-		super("Partition", input.getOutputType(), input.getParallelism());
-		this.input = input;
-		this.partitioner = partitioner;
-	}
-
-	/**
-	 * Returns the input {@code StreamTransformation} of this {@code SinkTransformation}.
-	 */
-	public StreamTransformation<T> getInput() {
-		return input;
-	}
-
-	/**
-	 * Returns the {@code StreamPartitioner} that must be used for partitioning the elements
-	 * of the input {@code StreamTransformation}.
-	 */
-	public StreamPartitioner<T> getPartitioner() {
-		return partitioner;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		result.addAll(input.getTransitivePredecessors());
-		return result;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation.");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
deleted file mode 100644
index a66b65a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This transformation represents a selection of only certain upstream elements. This must
- * follow a {@link org.apache.flink.streaming.api.transformations.SplitTransformation} that
- * splits elements into several logical streams with assigned names.
- *
- * <p>
- * This does not create a physical operation, it only affects how upstream operations are
- * connected to downstream operations.
- *
- * @param <T> The type of the elements that result from this {@code SelectTransformation}
- */
-public class SelectTransformation<T> extends StreamTransformation<T> {
-	
-	private final StreamTransformation<T> input;
-	private final List<String> selectedNames;
-
-	/**
-	 * Creates a new {@code SelectionTransformation} from the given input that only selects
-	 * the streams with the selected names.
-	 *
-	 * @param input The input {@code StreamTransformation}
-	 * @param selectedNames The names from the upstream {@code SplitTransformation} that this
-	 *                      {@code SelectTransformation} selects.
-	 */
-	public SelectTransformation(StreamTransformation<T> input,
-			List<String> selectedNames) {
-		super("Select", input.getOutputType(), input.getParallelism());
-		this.input = input;
-		this.selectedNames = selectedNames;
-	}
-
-	/**
-	 * Returns the input {@code StreamTransformation}.
-	 */
-	public StreamTransformation<T> getInput() {
-		return input;
-	}
-
-	/**
-	 * Returns the names of the split streams that this {@code SelectTransformation} selects.
-	 */
-	public List<String> getSelectedNames() {
-		return selectedNames;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		result.addAll(input.getTransitivePredecessors());
-		return result;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		throw new UnsupportedOperationException("Cannot set chaining strategy on Select Transformation.");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
deleted file mode 100644
index 84ad6db..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.StreamSink;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This Transformation represents a Sink.
- *
- * @param <T> The type of the elements in the input {@code SinkTransformation}
- */
-public class SinkTransformation<T> extends StreamTransformation<Object> {
-
-	private final StreamTransformation<T> input;
-
-	private final StreamSink<T> operator;
-
-	// We need this because sinks can also have state that is partitioned by key
-	private KeySelector<T, ?> stateKeySelector;
-	
-	private TypeInformation<?> stateKeyType;
-
-	/**
-	 * Creates a new {@code SinkTransformation} from the given input {@code StreamTransformation}.
-	 *
-	 * @param input The input {@code StreamTransformation}
-	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
-	 * @param operator The sink operator
-	 * @param parallelism The parallelism of this {@code SinkTransformation}
-	 */
-	public SinkTransformation(
-			StreamTransformation<T> input,
-			String name,
-			StreamSink<T> operator,
-			int parallelism) {
-		super(name, TypeExtractor.getForClass(Object.class), parallelism);
-		this.input = input;
-		this.operator = operator;
-	}
-
-	/**
-	 * Returns the input {@code StreamTransformation} of this {@code SinkTransformation}.
-	 */
-	public StreamTransformation<T> getInput() {
-		return input;
-	}
-
-	/**
-	 * Returns the {@link StreamSink} that is the operator of this {@code SinkTransformation}.
-	 */
-	public StreamSink<T> getOperator() {
-		return operator;
-	}
-
-	/**
-	 * Sets the {@link KeySelector} that must be used for partitioning keyed state of this Sink.
-	 *
-	 * @param stateKeySelector The {@code KeySelector} to set
-	 */
-	public void setStateKeySelector(KeySelector<T, ?> stateKeySelector) {
-		this.stateKeySelector = stateKeySelector;
-	}
-
-	/**
-	 * Returns the {@code KeySelector} that must be used for partitioning keyed state in this
-	 * Sink.
-	 *
-	 * @see #setStateKeySelector
-	 */
-	public KeySelector<T, ?> getStateKeySelector() {
-		return stateKeySelector;
-	}
-
-	public void setStateKeyType(TypeInformation<?> stateKeyType) {
-		this.stateKeyType = stateKeyType;
-	}
-
-	public TypeInformation<?> getStateKeyType() {
-		return stateKeyType;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		result.addAll(input.getTransitivePredecessors());
-		return result;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		operator.setChainingStrategy(strategy);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
deleted file mode 100644
index 9835606..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.transformations;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.StreamSource;
-
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * This represents a Source. This does not actually transform anything since it has no inputs but
- * it is the root {@code StreamTransformation} of any topology.
- *
- * @param <T> The type of the elements that this source produces
- */
-public class SourceTransformation<T> extends StreamTransformation<T> {
-
-	private final StreamSource<T> operator;
-
-	/**
-	 * Creates a new {@code SourceTransformation} from the given operator.
-	 *
-	 * @param name The name of the {@code SourceTransformation}, this will be shown in Visualizations and the Log
-	 * @param operator The {@code StreamSource} that is the operator of this Transformation
-	 * @param outputType The type of the elements produced by this {@code SourceTransformation}
-	 * @param parallelism The parallelism of this {@code SourceTransformation}
-	 */
-	public SourceTransformation(
-			String name,
-			StreamSource<T> operator,
-			TypeInformation<T> outputType,
-			int parallelism) {
-		super(name, outputType, parallelism);
-		this.operator = operator;
-	}
-
-	/**
-	 * Returns the {@code StreamSource}, the operator of this {@code SourceTransformation}.
-	 */
-	public StreamSource<T> getOperator() {
-		return operator;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		return Collections.<StreamTransformation<?>>singleton(this);
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		operator.setChainingStrategy(strategy);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
deleted file mode 100644
index 96c1c9e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This transformation represents a split of one
- * {@link org.apache.flink.streaming.api.datastream.DataStream} into several {@code DataStreams}
- * using an {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}.
- *
- * <p>
- * This does not create a physical operation, it only affects how upstream operations are
- * connected to downstream operations.
- *
- * @param <T> The type of the elements that result from this {@code SplitTransformation}
- */
-public class SplitTransformation<T> extends StreamTransformation<T> {
-
-	private final StreamTransformation<T> input;
-
-	private final OutputSelector<T> outputSelector;
-
-	/**
-	 * Creates a new {@code SplitTransformation} from the given input and {@code OutputSelector}.
-	 *
-	 * @param input The input {@code StreamTransformation}
-	 * @param outputSelector The output selector
-	 */
-	public SplitTransformation(StreamTransformation<T> input,
-			OutputSelector<T> outputSelector) {
-		super("Split", input.getOutputType(), input.getParallelism());
-		this.input = input;
-		this.outputSelector = outputSelector;
-	}
-
-	/**
-	 * Returns the input {@code StreamTransformation}.
-	 */
-	public StreamTransformation<T> getInput() {
-		return input;
-	}
-
-	/**
-	 * Returns the {@code OutputSelector}
-	 */
-	public OutputSelector<T> getOutputSelector() {
-		return outputSelector;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		result.addAll(input.getTransitivePredecessors());
-		return result;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
deleted file mode 100644
index 4e6dc42..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
+++ /dev/null
@@ -1,321 +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.transformations;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.MissingTypeInfo;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-
-import java.util.Collection;
-
-/**
- * A {@code StreamTransformation} represents the operation that creates a
- * {@link org.apache.flink.streaming.api.datastream.DataStream}. Every
- * {@link org.apache.flink.streaming.api.datastream.DataStream} has an underlying
- * {@code StreamTransformation} that is the origin of said DataStream.
- *
- * <p>
- * API operations such as {@link org.apache.flink.streaming.api.datastream.DataStream#map} create
- * a tree of {@code StreamTransformation}s underneath. When the stream program is to be executed this
- * graph is translated to a {@link StreamGraph} using
- * {@link org.apache.flink.streaming.api.graph.StreamGraphGenerator}.
- *
- * <p>
- * A {@code StreamTransformation} does not necessarily correspond to a physical operation
- * at runtime. Some operations are only logical concepts. Examples of this are union,
- * split/select data stream, partitioning.
- *
- * <p>
- * The following graph of {@code StreamTransformations}:
- *
- * <pre>
- *   Source              Source        
- *      +                   +           
- *      |                   |           
- *      v                   v           
- *  Rebalance          HashPartition    
- *      +                   +           
- *      |                   |           
- *      |                   |           
- *      +------>Union<------+           
- *                +                     
- *                |                     
- *                v                     
- *              Split                   
- *                +                     
- *                |                     
- *                v                     
- *              Select                  
- *                +                     
- *                v                     
- *               Map                    
- *                +                     
- *                |                     
- *                v                     
- *              Sink 
- * </pre>
- *
- * Would result in this graph of operations at runtime:
- *
- * <pre>
- *  Source              Source
- *    +                   +
- *    |                   |
- *    |                   |
- *    +------->Map<-------+
- *              +
- *              |
- *              v
- *             Sink
- * </pre>
- *
- * The information about partitioning, union, split/select end up being encoded in the edges
- * that connect the sources to the map operation.
- *
- * @param <T> The type of the elements that result from this {@code StreamTransformation}
- */
-public abstract class StreamTransformation<T> {
-
-	// This is used to assign a unique ID to every StreamTransformation
-	protected static Integer idCounter = 0;
-	public static int getNewNodeId() {
-		idCounter++;
-		return idCounter;
-	}
-
-	protected final int id;
-
-	protected String name;
-
-	protected TypeInformation<T> outputType;
-	// This is used to handle MissingTypeInfo. As long as the outputType has not been queried
-	// it can still be changed using setOutputType(). Afterwards an exception is thrown when
-	// trying to change the output type.
-	protected boolean typeUsed;
-
-	private int parallelism;
-
-	protected long bufferTimeout = -1;
-
-	protected StreamGraph.ResourceStrategy resourceStrategy = StreamGraph.ResourceStrategy.DEFAULT;
-
-	/**
-	 * Creates a new {@code StreamTransformation} with the given name, output type and parallelism.
-	 *
-	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
-	 * @param outputType The output type of this {@code StreamTransformation}
-	 * @param parallelism The parallelism of this {@code StreamTransformation}
-	 */
-	public StreamTransformation(String name, TypeInformation<T> outputType, int parallelism) {
-		this.id = getNewNodeId();
-		this.name = Preconditions.checkNotNull(name);
-		this.outputType = outputType;
-		this.parallelism = parallelism;
-	}
-
-	/**
-	 * Returns the unique ID of this {@code StreamTransformation}.
-	 */
-	public int getId() {
-		return id;
-	}
-
-	/**
-	 * Changes the name of this {@code StreamTransformation}.
-	 */
-	public void setName(String name) {
-		this.name = name;
-	}
-
-	/**
-	 * Returns the name of this {@code StreamTransformation}.
-	 */
-	public String getName() {
-		return name;
-	}
-
-	/**
-	 * Returns the parallelism of this {@code StreamTransformation}
-	 */
-	public int getParallelism() {
-		return parallelism;
-	}
-
-	/**
-	 * Sets the parallelism of this {@code StreamTransformation}
-	 * @param parallelism The new parallelism to set on this {@code StreamTransformation}
-	 */
-	public void setParallelism(int parallelism) {
-		Preconditions.checkArgument(parallelism > 0, "Parallelism must be bigger than zero.");
-		this.parallelism = parallelism;
-	}
-
-	/**
-	 * Tries to fill in the type information. Type information can be filled in
-	 * later when the program uses a type hint. This method checks whether the
-	 * type information has ever been accessed before and does not allow
-	 * modifications if the type was accessed already. This ensures consistency
-	 * by making sure different parts of the operation do not assume different
-	 * type information.
-	 *
-	 * @param outputType The type information to fill in.
-	 *
-	 * @throws IllegalStateException Thrown, if the type information has been accessed before.
-	 */
-	public void setOutputType(TypeInformation<T> outputType) {
-		if (typeUsed) {
-			throw new IllegalStateException(
-					"TypeInformation cannot be filled in for the type after it has been used. "
-							+ "Please make sure that the type info hints are the first call after"
-							+ " the transformation function, "
-							+ "before any access to types or semantic properties, etc.");
-		}
-		this.outputType = outputType;
-	}
-
-	/**
-	 * Returns the output type of this {@code StreamTransformation} as a {@link TypeInformation}. Once
-	 * this is used once the output type cannot be changed anymore using {@link #setOutputType}.
-	 *
-	 * @return The output type of this {@code StreamTransformation}
-	 */
-	public TypeInformation<T> getOutputType() {
-		if (outputType instanceof MissingTypeInfo) {
-			MissingTypeInfo typeInfo = (MissingTypeInfo) this.outputType;
-			throw new InvalidTypesException(
-					"The return type of function '"
-							+ typeInfo.getFunctionName()
-							+ "' could not be determined automatically, due to type erasure. "
-							+ "You can give type information hints by using the returns(...) "
-							+ "method on the result of the transformation call, or by letting "
-							+ "your function implement the 'ResultTypeQueryable' "
-							+ "interface.", typeInfo.getTypeException());
-		}
-		typeUsed = true;
-		return this.outputType;
-	}
-
-	/**
-	 * Sets the chaining strategy of this {@code StreamTransformation}.
-	 */
-	public abstract void setChainingStrategy(ChainingStrategy strategy);
-
-	/**
-	 * Set the buffer timeout of this {@code StreamTransformation}. The timeout is used when
-	 * sending elements over the network. The timeout specifies how long a network buffer
-	 * should be kept waiting before sending. A higher timeout means that more elements will
-	 * be sent in one buffer, this increases throughput. The latency, however, is negatively
-	 * affected by a higher timeout.
-	 */
-	public void setBufferTimeout(long bufferTimeout) {
-		this.bufferTimeout = bufferTimeout;
-	}
-
-	/**
-	 * Returns the buffer timeout of this {@code StreamTransformation}.
-	 *
-	 * <p>
-	 * {@see #setBufferTimeout}
-	 */
-	public long getBufferTimeout() {
-		return bufferTimeout;
-	}
-
-	/**
-	 * Sets the {@link org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy} of this
-	 * {@code StreamTransformation}. The resource strategy is used when scheduling operations on actual
-	 * workers when transforming the StreamTopology to an
-	 * {@link org.apache.flink.runtime.executiongraph.ExecutionGraph}.
-	 */
-	public void setResourceStrategy(StreamGraph.ResourceStrategy resourceStrategy) {
-		this.resourceStrategy = resourceStrategy;
-	}
-
-	/**
-	 * Returns the {@code ResourceStrategy} of this {@code StreamTransformation}.
-	 *
-	 * <p>
-	 * {@see #setResourceStrategy}
-	 */
-	public StreamGraph.ResourceStrategy getResourceStrategy() {
-		return resourceStrategy;
-	}
-
-	/**
-	 * Returns all transitive predecessor {@code StreamTransformation}s of this {@code StreamTransformation}. This
-	 * is, for example, used when determining whether a feedback edge of an iteration
-	 * actually has the iteration head as a predecessor.
-	 *
-	 * @return The list of transitive predecessors.
-	 */
-	public abstract Collection<StreamTransformation<?>> getTransitivePredecessors();
-
-	@Override
-	public String toString() {
-		return getClass().getSimpleName() + "{" +
-				"id=" + id +
-				", name='" + name + '\'' +
-				", outputType=" + outputType +
-				", parallelism=" + parallelism +
-				'}';
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (!(o instanceof StreamTransformation)) {
-			return false;
-		}
-
-		StreamTransformation<?> that = (StreamTransformation<?>) o;
-
-		if (bufferTimeout != that.bufferTimeout) {
-			return false;
-		}
-		if (id != that.id) {
-			return false;
-		}
-		if (parallelism != that.parallelism) {
-			return false;
-		}
-		if (!name.equals(that.name)) {
-			return false;
-		}
-		if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) {
-			return false;
-		}
-		return resourceStrategy == that.resourceStrategy;
-	}
-
-	@Override
-	public int hashCode() {
-		int result = id;
-		result = 31 * result + name.hashCode();
-		result = 31 * result + (outputType != null ? outputType.hashCode() : 0);
-		result = 31 * result + parallelism;
-		result = 31 * result + (int) (bufferTimeout ^ (bufferTimeout >>> 32));
-		result = 31 * result + resourceStrategy.hashCode();
-		return result;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
deleted file mode 100644
index 30f0733..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
+++ /dev/null
@@ -1,116 +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.transformations;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-
-import java.util.Collection;
-import java.util.List;
-
-/**
- * This Transformation represents the application of a
- * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator} to two input
- * {@code StreamTransformations}. The result is again only one stream.
- *
- * @param <IN1> The type of the elements in the first input {@code StreamTransformation}
- * @param <IN2> The type of the elements in the second input {@code StreamTransformation}
- * @param <OUT> The type of the elements that result from this {@code TwoInputTransformation}
- */
-public class TwoInputTransformation<IN1, IN2, OUT> extends StreamTransformation<OUT> {
-
-	private final StreamTransformation<IN1> input1;
-	private final StreamTransformation<IN2> input2;
-
-	private final TwoInputStreamOperator<IN1, IN2, OUT> operator;
-
-	/**
-	 * Creates a new {@code TwoInputTransformation} from the given inputs and operator.
-	 *
-	 * @param input1 The first input {@code StreamTransformation}
-	 * @param input2 The second input {@code StreamTransformation}
-	 * @param name The name of the {@code StreamTransformation}, this will be shown in Visualizations and the Log
-	 * @param operator The {@code TwoInputStreamOperator}
-	 * @param outputType The type of the elements produced by this Transformation
-	 * @param parallelism The parallelism of this Transformation
-	 */
-	public TwoInputTransformation(
-			StreamTransformation<IN1> input1,
-			StreamTransformation<IN2> input2,
-			String name,
-			TwoInputStreamOperator<IN1, IN2, OUT> operator,
-			TypeInformation<OUT> outputType,
-			int parallelism) {
-		super(name, outputType, parallelism);
-		this.input1 = input1;
-		this.input2 = input2;
-		this.operator = operator;
-	}
-
-	/**
-	 * Returns the first input {@code StreamTransformation} of this {@code TwoInputTransformation}.
-	 */
-	public StreamTransformation<IN1> getInput1() {
-		return input1;
-	}
-
-	/**
-	 * Returns the first input {@code StreamTransformation} of this {@code TwoInputTransformation}.
-	 */
-	public StreamTransformation<IN2> getInput2() {
-		return input2;
-	}
-
-	/**
-	 * Returns the {@code TypeInformation} for the elements from the first input.
-	 */
-	public TypeInformation<IN1> getInputType1() {
-		return input1.getOutputType();
-	}
-
-	/**
-	 * Returns the {@code TypeInformation} for the elements from the first input.
-	 */
-	public TypeInformation<IN2> getInputType2() {
-		return input2.getOutputType();
-	}
-
-	/**
-	 * Returns the {@code TwoInputStreamOperator} of this Transformation.
-	 */
-	public TwoInputStreamOperator<IN1, IN2, OUT> getOperator() {
-		return operator;
-	}
-
-	@Override
-	public Collection<StreamTransformation<?>> getTransitivePredecessors() {
-		List<StreamTransformation<?>> result = Lists.newArrayList();
-		result.add(this);
-		result.addAll(input1.getTransitivePredecessors());
-		result.addAll(input2.getTransitivePredecessors());
-		return result;
-	}
-
-	@Override
-	public final void setChainingStrategy(ChainingStrategy strategy) {
-		operator.setChainingStrategy(strategy);
-	}
-
-}


[45/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml
deleted file mode 100644
index d373d63..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/test/resources/NiFi_Flink.xml
+++ /dev/null
@@ -1,16 +0,0 @@
-<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements. See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License. You may obtain a copy of the License at
-  http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<template><description></description><name>NiFi_Flink</name><snippet><connections><id>34acfdda-dd21-48c0-8779-95d0e258f5cb</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>8b8b6a2f-ee24-4f32-a178-248f3a3f5482</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><selectedRelationships>success</selectedRelationships><source><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>769242e5-ee04-4656-a684-ca661a18eed6</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>59574e3b-1ba7-4343-b265-af1b67923a85</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThresh
 old>0</backPressureObjectThreshold><destination><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>48042218-a51e-45c7-bd30-2290bba8b191</id><type>OUTPUT_PORT</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><selectedRelationships>success</selectedRelationships><source><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>8b8b6a2f-ee24-4f32-a178-248f3a3f5482</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>46c9343f-f732-4e2d-98e1-13caab5d2f5e</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>e3a9026f-dae1-42ca-851c-02d9fda22094</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><source><groupI
 d>0f854f2b-239f-45f0-bfed-48b5b23f7928</groupId><id>cd8c5227-cfa9-4603-9472-b2234d7bd741</id><type>INPUT_PORT</type></source><zIndex>0</zIndex></connections><inputPorts><id>cd8c5227-cfa9-4603-9472-b2234d7bd741</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>395.0</x><y>520.0</y></position><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><name>Data from Flink</name><state>RUNNING</state><transmitting>false</transmitting><type>INPUT_PORT</type></inputPorts><outputPorts><id>48042218-a51e-45c7-bd30-2290bba8b191</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>1616.0</x><y>259.0</y></position><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><name>Data for Flink</name><state>RUNNING</state><transmitting>false</transmitting><type>OUTPUT_PORT</type></outputPorts><processors><id>769242e5-ee04-4656-a684-ca661a18eed6</id><parentGroupId>0f854f2b-239f-45f0-bfed-48
 b5b23f7928</parentGroupId><position><x>389.0</x><y>231.0</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><defaultConcurrentTasks><entry><key>TIMER_DRIVEN</key><value>1</value></entry><entry><key>EVENT_DRIVEN</key><value>0</value></entry><entry><key>CRON_DRIVEN</key><value>1</value></entry></defaultConcurrentTasks><defaultSchedulingPeriod><entry><key>TIMER_DRIVEN</key><value>0 sec</value></entry><entry><key>CRON_DRIVEN</key><value>* * * * * ?</value></entry></defaultSchedulingPeriod><descriptors><entry><key>File Size</key><value><description>The size of the file that will be used</description><displayName>File Size</displayName><dynamic>false</dynamic><name>File Size</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Batch Size</key><value><defaultValue>1</defaultValue><description>The number of FlowFiles to be transferr
 ed in each invocation</description><displayName>Batch Size</displayName><dynamic>false</dynamic><name>Batch Size</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Data Format</key><value><allowableValues><displayName>Binary</displayName><value>Binary</value></allowableValues><allowableValues><displayName>Text</displayName><value>Text</value></allowableValues><defaultValue>Binary</defaultValue><description>Specifies whether the data should be Text or Binary</description><displayName>Data Format</displayName><dynamic>false</dynamic><name>Data Format</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Unique FlowFiles</key><value><allowableValues><displayName>true</displayName><value>true</value></allowableValues><allowableValues><displayName>false</displayName><value>false</value></allowableValues><defaultValue>false</defaultValue><description>If true, ea
 ch FlowFile that is generated will be unique. If false, a random value will be generated and all FlowFiles will get the same content but this offers much higher throughput</description><displayName>Unique FlowFiles</displayName><dynamic>false</dynamic><name>Unique FlowFiles</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry></descriptors><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>File Size</key><value>1 b</value></entry><entry><key>Batch Size</key><value>1</value></entry><entry><key>Data Format</key><value>Binary</value></entry><entry><key>Unique FlowFiles</key><value>false</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>2 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GenerateFlowFile</name><relationships><autoTerminate>false</autoTerminate><description></des
 cription><name>success</name></relationships><state>STOPPED</state><style/><supportsEventDriven>false</supportsEventDriven><supportsParallelProcessing>true</supportsParallelProcessing><type>org.apache.nifi.processors.standard.GenerateFlowFile</type></processors><processors><id>e3a9026f-dae1-42ca-851c-02d9fda22094</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>826.0</x><y>499.0</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><defaultConcurrentTasks><entry><key>TIMER_DRIVEN</key><value>1</value></entry><entry><key>EVENT_DRIVEN</key><value>0</value></entry><entry><key>CRON_DRIVEN</key><value>1</value></entry></defaultConcurrentTasks><defaultSchedulingPeriod><entry><key>TIMER_DRIVEN</key><value>0 sec</value></entry><entry><key>CRON_DRIVEN</key><value>* * * * * ?</value></entry></defaultSchedulingPeriod><descriptors><entry><key>Log Level</key><value><al
 lowableValues><displayName>trace</displayName><value>trace</value></allowableValues><allowableValues><displayName>debug</displayName><value>debug</value></allowableValues><allowableValues><displayName>info</displayName><value>info</value></allowableValues><allowableValues><displayName>warn</displayName><value>warn</value></allowableValues><allowableValues><displayName>error</displayName><value>error</value></allowableValues><defaultValue>info</defaultValue><description>The Log Level to use when logging the Attributes</description><displayName>Log Level</displayName><dynamic>false</dynamic><name>Log Level</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Log Payload</key><value><allowableValues><displayName>true</displayName><value>true</value></allowableValues><allowableValues><displayName>false</displayName><value>false</value></allowableValues><defaultValue>false</defaultValue><description>If true, the FlowFile's p
 ayload will be logged, in addition to its attributes; otherwise, just the Attributes will be logged.</description><displayName>Log Payload</displayName><dynamic>false</dynamic><name>Log Payload</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Attributes to Log</key><value><description>A comma-separated list of Attributes to Log. If not specified, all attributes will be logged.</description><displayName>Attributes to Log</displayName><dynamic>false</dynamic><name>Attributes to Log</name><required>false</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Attributes to Ignore</key><value><description>A comma-separated list of Attributes to ignore. If not specified, no attributes will be ignored.</description><displayName>Attributes to Ignore</displayName><dynamic>false</dynamic><name>Attributes to Ignore</name><required>false</required><sensitive>false</sensitive><supportsEl>
 false</supportsEl></value></entry><entry><key>Log prefix</key><value><description>Log prefix appended to the log lines. It helps to distinguish the output of multiple LogAttribute processors.</description><displayName>Log prefix</displayName><dynamic>false</dynamic><name>Log prefix</name><required>false</required><sensitive>false</sensitive><supportsEl>true</supportsEl></value></entry></descriptors><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Log Level</key></entry><entry><key>Log Payload</key><value>true</value></entry><entry><key>Attributes to Log</key></entry><entry><key>Attributes to Ignore</key></entry><entry><key>Log prefix</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>LogAttribute</name><relationships><autoTerminate>true</autoTerminate><description>All FlowFil
 es are routed to this relationship</description><name>success</name></relationships><state>RUNNING</state><style/><supportsEventDriven>true</supportsEventDriven><supportsParallelProcessing>true</supportsParallelProcessing><type>org.apache.nifi.processors.standard.LogAttribute</type></processors><processors><id>8b8b6a2f-ee24-4f32-a178-248f3a3f5482</id><parentGroupId>0f854f2b-239f-45f0-bfed-48b5b23f7928</parentGroupId><position><x>1000.0</x><y>231.0</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><defaultConcurrentTasks><entry><key>TIMER_DRIVEN</key><value>1</value></entry><entry><key>EVENT_DRIVEN</key><value>0</value></entry><entry><key>CRON_DRIVEN</key><value>1</value></entry></defaultConcurrentTasks><defaultSchedulingPeriod><entry><key>TIMER_DRIVEN</key><value>0 sec</value></entry><entry><key>CRON_DRIVEN</key><value>* * * * * ?</value></entry></defaultSchedulingPeriod><descriptors><e
 ntry><key>Regular Expression</key><value><defaultValue>(?s:^.*$)</defaultValue><description>The Regular Expression to search for in the FlowFile content</description><displayName>Regular Expression</displayName><dynamic>false</dynamic><name>Regular Expression</name><required>true</required><sensitive>false</sensitive><supportsEl>true</supportsEl></value></entry><entry><key>Replacement Value</key><value><defaultValue>$1</defaultValue><description>The value to replace the regular expression with. Back-references to Regular Expression capturing groups are supported, but back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value.</description><displayName>Replacement Value</displayName><dynamic>false</dynamic><name>Replacement Value</name><required>true</required><sensitive>false</sensitive><supportsEl>true</supportsEl></value></entry><entry><key>Character Set</key><value><defaultValue>UTF-8</defaultValue><description>The
  Character Set in which the file is encoded</description><displayName>Character Set</displayName><dynamic>false</dynamic><name>Character Set</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Maximum Buffer Size</key><value><defaultValue>1 MB</defaultValue><description>Specifies the maximum amount of data to buffer (per file or per line, depending on the Evaluation Mode) in order to apply the regular expressions. If 'Entire Text' (in Evaluation Mode) is selected and the FlowFile is larger than this value, the FlowFile will be routed to 'failure'. In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. This value is ignored and the buffer is not used if 'Regular Expression' is set to '.*'</description><displayName>Maximum Buffer 
 Size</displayName><dynamic>false</dynamic><name>Maximum Buffer Size</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry><entry><key>Evaluation Mode</key><value><allowableValues><displayName>Line-by-Line</displayName><value>Line-by-Line</value></allowableValues><allowableValues><displayName>Entire text</displayName><value>Entire text</value></allowableValues><defaultValue>Entire text</defaultValue><description>Evaluate the 'Regular Expression' against each line (Line-by-Line) or buffer the entire file into memory (Entire Text) and then evaluate the 'Regular Expression'.</description><displayName>Evaluation Mode</displayName><dynamic>false</dynamic><name>Evaluation Mode</name><required>true</required><sensitive>false</sensitive><supportsEl>false</supportsEl></value></entry></descriptors><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Regular Expression</key><value>(?s:^.*$)</value><
 /entry><entry><key>Replacement Value</key><value>blah blah</value></entry><entry><key>Character Set</key><value>UTF-8</value></entry><entry><key>Maximum Buffer Size</key><value>1 MB</value></entry><entry><key>Evaluation Mode</key><value>Entire text</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>ReplaceText</name><relationships><autoTerminate>true</autoTerminate><description>FlowFiles that could not be updated are routed to this relationship</description><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><description>FlowFiles that have been successfully updated are routed to this relationship, as well as FlowFiles whose content does not match the given Regular Expression</description><name>success</name></relationships><state>RUNNING</state><style/><supportsEventDriven>true</supports
 EventDriven><supportsParallelProcessing>true</supportsParallelProcessing><type>org.apache.nifi.processors.standard.ReplaceText</type></processors></snippet><timestamp>09/30/2015 09:10:38 EDT</timestamp></template>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
deleted file mode 100644
index a361920..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/pom.xml
+++ /dev/null
@@ -1,59 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-rabbitmq</artifactId>
-	<name>flink-connector-rabbitmq</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<rabbitmq.version>3.3.1</rabbitmq.version>
-	</properties>
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>com.rabbitmq</groupId>
-			<artifactId>amqp-client</artifactId>
-			<version>${rabbitmq.version}</version>
-		</dependency>
-
-	</dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
deleted file mode 100644
index fa729d6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ /dev/null
@@ -1,111 +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.connectors.rabbitmq;
-
-import java.io.IOException;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.rabbitmq.client.Channel;
-import com.rabbitmq.client.Connection;
-import com.rabbitmq.client.ConnectionFactory;
-
-public class RMQSink<IN> extends RichSinkFunction<IN> {
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(RMQSink.class);
-
-	private String QUEUE_NAME;
-	private String HOST_NAME;
-	private transient ConnectionFactory factory;
-	private transient Connection connection;
-	private transient Channel channel;
-	private SerializationSchema<IN, byte[]> schema;
-
-	public RMQSink(String HOST_NAME, String QUEUE_NAME, SerializationSchema<IN, byte[]> schema) {
-		this.HOST_NAME = HOST_NAME;
-		this.QUEUE_NAME = QUEUE_NAME;
-		this.schema = schema;
-	}
-
-	/**
-	 * Initializes the connection to RMQ.
-	 */
-	public void initializeConnection() {
-		factory = new ConnectionFactory();
-		factory.setHost(HOST_NAME);
-		try {
-			connection = factory.newConnection();
-			channel = connection.createChannel();
-			channel.queueDeclare(QUEUE_NAME, false, false, false, null);
-
-		} catch (IOException e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	/**
-	 * Called when new data arrives to the sink, and forwards it to RMQ.
-	 * 
-	 * @param value
-	 *            The incoming data
-	 */
-	@Override
-	public void invoke(IN value) {
-		try {
-			byte[] msg = schema.serialize(value);
-
-			channel.basicPublish("", QUEUE_NAME, null, msg);
-
-		} catch (IOException e) {
-			if (LOG.isErrorEnabled()) {
-				LOG.error("Cannot send RMQ message {} at {}", QUEUE_NAME, HOST_NAME);
-			}
-		}
-
-	}
-
-	/**
-	 * Closes the connection.
-	 */
-	private void closeChannel() {
-		try {
-			channel.close();
-			connection.close();
-		} catch (IOException e) {
-			throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME
-					+ " at " + HOST_NAME, e);
-		}
-
-	}
-
-	@Override
-	public void open(Configuration config) {
-		initializeConnection();
-	}
-
-	@Override
-	public void close() {
-		closeChannel();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
deleted file mode 100644
index b18b8d8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.rabbitmq;
-
-import java.io.IOException;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.source.ConnectorSource;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import com.rabbitmq.client.Channel;
-import com.rabbitmq.client.Connection;
-import com.rabbitmq.client.ConnectionFactory;
-import com.rabbitmq.client.QueueingConsumer;
-
-public class RMQSource<OUT> extends ConnectorSource<OUT> {
-	private static final long serialVersionUID = 1L;
-
-	private final String QUEUE_NAME;
-	private final String HOST_NAME;
-
-	private transient ConnectionFactory factory;
-	private transient Connection connection;
-	private transient Channel channel;
-	private transient QueueingConsumer consumer;
-	private transient QueueingConsumer.Delivery delivery;
-
-	private transient volatile boolean running;
-
-	public RMQSource(String HOST_NAME, String QUEUE_NAME,
-			DeserializationSchema<OUT> deserializationSchema) {
-		super(deserializationSchema);
-		this.HOST_NAME = HOST_NAME;
-		this.QUEUE_NAME = QUEUE_NAME;
-	}
-
-	/**
-	 * Initializes the connection to RMQ.
-	 */
-	private void initializeConnection() {
-		factory = new ConnectionFactory();
-		factory.setHost(HOST_NAME);
-		try {
-			connection = factory.newConnection();
-			channel = connection.createChannel();
-			channel.queueDeclare(QUEUE_NAME, false, false, false, null);
-			consumer = new QueueingConsumer(channel);
-			channel.basicConsume(QUEUE_NAME, true, consumer);
-		} catch (IOException e) {
-			throw new RuntimeException("Cannot create RMQ connection with " + QUEUE_NAME + " at "
-					+ HOST_NAME, e);
-		}
-	}
-
-	@Override
-	public void open(Configuration config) throws Exception {
-		initializeConnection();
-		running = true;
-	}
-
-	@Override
-	public void close() throws Exception {
-		super.close();
-		try {
-			connection.close();
-		} catch (IOException e) {
-			throw new RuntimeException("Error while closing RMQ connection with " + QUEUE_NAME
-					+ " at " + HOST_NAME, e);
-		}
-	}
-
-	@Override
-	public void run(SourceContext<OUT> ctx) throws Exception {
-		while (running) {
-			delivery = consumer.nextDelivery();
-
-			OUT result = schema.deserialize(delivery.getBody());
-			if (schema.isEndOfStream(result)) {
-				break;
-			}
-
-			ctx.collect(result);
-		}
-	}
-
-	@Override
-	public void cancel() {
-		running = false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
deleted file mode 100644
index 1f85862..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQTopology.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.rabbitmq;
-
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-
-public class RMQTopology {
-
-	public static void main(String[] args) throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-
-		@SuppressWarnings("unused")
-		DataStreamSink<String> dataStream1 = env.addSource(
-				new RMQSource<String>("localhost", "hello", new SimpleStringSchema())).print();
-
-		@SuppressWarnings("unused")
-		DataStreamSink<String> dataStream2 = env.fromElements("one", "two", "three", "four", "five",
-				"q").addSink(
-				new RMQSink<String>("localhost", "hello", new StringToByteSerializer()));
-
-		env.execute();
-	}
-
-	public static class StringToByteSerializer implements SerializationSchema<String, byte[]> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public byte[] serialize(String element) {
-			return element.getBytes();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/pom.xml
deleted file mode 100644
index f1ef0f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/pom.xml
+++ /dev/null
@@ -1,97 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-twitter</artifactId>
-	<name>flink-connector-twitter</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>com.twitter</groupId>
-			<artifactId>hbc-core</artifactId>
-			<version>2.2.0</version>
-		</dependency>
-		
-		<dependency>
-			<groupId>org.apache.sling</groupId>
-			<artifactId>org.apache.sling.commons.json</artifactId>
-			<version>2.0.6</version>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<!-- Override artifactSet configuration to build fat-jar with all dependencies packed. -->
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<configuration>
-							<artifactSet>
-								<includes combine.children="append">
-									<!-- We include all dependencies that transitively depend on guava -->
-									<include>com.twitter:hbc-core</include>
-									<include>com.twitter:joauth</include>
-								</includes>
-							</artifactSet>
-							<transformers>
-								<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
-							</transformers>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
deleted file mode 100644
index 0f16541..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParseFlatMap.java
+++ /dev/null
@@ -1,144 +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.connectors.json;
-
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.sling.commons.json.JSONException;
-
-/**
- * Abstract class derived from {@link RichFlatMapFunction} to handle JSON files.
- * 
- * @param <IN>
- *            Type of the input elements.
- * @param <OUT>
- *            Type of the returned elements.
- */
-public abstract class JSONParseFlatMap<IN, OUT> extends RichFlatMapFunction<IN, OUT> {
-
-	private static final long serialVersionUID = 1L;
-
-	// private static final Log LOG = LogFactory.getLog(JSONParseFlatMap.class);
-
-	/**
-	 * Get the value object associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public Object get(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).get("retValue");
-	}
-
-	/**
-	 * Get the boolean value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public boolean getBoolean(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getBoolean("retValue");
-	}
-
-	/**
-	 * Get the double value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public double getDouble(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getDouble("retValue");
-	}
-
-	/**
-	 * Get the int value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public int getInt(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getInt("retValue");
-	}
-
-	/**
-	 * Get the long value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public long getLong(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-
-		return parser.parse(field).getLong("retValue");
-	}
-	
-	/**
-	 * Get the String value associated with a key form a JSON code. It can find
-	 * embedded fields, too.
-	 * 
-	 * @param jsonText
-	 *            JSON String in which the field is searched.
-	 * @param field
-	 *            The key whose value is searched for.
-	 * @return The object associated with the field.
-	 * @throws JSONException
-	 *             If the field is not found.
-	 */
-	public String getString(String jsonText, String field) throws JSONException {
-		JSONParser parser = new JSONParser(jsonText);
-		
-		return parser.parse(field).getString("retValue");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
deleted file mode 100644
index c1eabbd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/json/JSONParser.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.json;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.StringTokenizer;
-
-import org.apache.sling.commons.json.JSONArray;
-import org.apache.sling.commons.json.JSONException;
-import org.apache.sling.commons.json.JSONObject;
-
-/**
- * A JSONParser contains a JSONObject and provides opportunity to access
- * embedded fields in JSON code.
- */
-public class JSONParser {
-
-	private JSONObject originalJO;
-	private String searchedfield;
-	private Object temp;
-
-	/**
-	 * Construct a JSONParser from a string. The string has to be a JSON code
-	 * from which we want to get a field.
-	 * 
-	 * @param jsonText
-	 *            A string which contains a JSON code. String representation of
-	 *            a JSON code.
-	 * @throws JSONException
-	 *             If there is a syntax error in the source string.
-	 */
-	public JSONParser(String jsonText) throws JSONException {
-		originalJO = new JSONObject(jsonText);
-	}
-
-	/**
-	 * 
-	 * Parse the JSON code passed to the constructor to find the given key.
-	 * 
-	 * @param key
-	 *            The key whose value is searched for.
-	 * @return A JSONObject which has only one field called "retValue" and the
-	 *         value associated to it is the searched value. The methods of
-	 *         JSONObject can be used to get the field value in a desired
-	 *         format.
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	public JSONObject parse(String key) throws JSONException {
-		initializeParser(key);
-		parsing();
-		return putResultInJSONObj();
-	}
-
-	/**
-	 * Prepare the fields of the class for the parsing
-	 * 
-	 * @param key
-	 *            The key whose value is searched for.
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	private void initializeParser(String key) throws JSONException {
-		searchedfield = key;
-		temp = new JSONObject(originalJO.toString());
-	}
-
-	/**
-	 * This function goes through the given field and calls the appropriate
-	 * functions to treat the units between the punctuation marks.
-	 * 
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	private void parsing() throws JSONException {
-		StringTokenizer st = new StringTokenizer(searchedfield, ".");
-		while (st.hasMoreTokens()) {
-			find(st.nextToken());
-		}
-	}
-
-	/**
-	 * Search for the next part of the field and update the state if it was
-	 * found.
-	 * 
-	 * @param nextToken
-	 *            The current part of the searched field.
-	 * @throws JSONException
-	 *             If the key is not found.
-	 */
-	private void find(String nextToken) throws JSONException {
-		if (endsWithBracket(nextToken)) {
-			treatAllBracket(nextToken);
-		} else {
-			temp = ((JSONObject) temp).get(nextToken);
-		}
-	}
-
-	/**
-	 * Determine whether the given string ends with a closing square bracket ']'
-	 * 
-	 * @param nextToken
-	 *            The current part of the searched field.
-	 * @return True if the given string ends with a closing square bracket ']'
-	 *         and false otherwise.
-	 */
-	private boolean endsWithBracket(String nextToken) {
-		return nextToken.substring(nextToken.length() - 1).endsWith("]");
-	}
-
-	/**
-	 * Handle (multidimensional) arrays. Treat the square bracket pairs one
-	 * after the other if necessary.
-	 * 
-	 * @param nextToken
-	 *            The current part of the searched field.
-	 * @throws JSONException
-	 *             If the searched element is not found.
-	 */
-	private void treatAllBracket(String nextToken) throws JSONException {
-		List<String> list = Arrays.asList(nextToken.split("\\["));
-		ListIterator<String> iter = list.listIterator();
-
-		temp = ((JSONObject) temp).get(iter.next());
-
-		while (iter.hasNext()) {
-			int index = Integer.parseInt(cutBracket(iter.next()));
-			temp = ((JSONArray) temp).get(index);
-		}
-	}
-
-	/**
-	 * Remove the last character of the string.
-	 * 
-	 * @param string
-	 *            String to modify.
-	 * @return The given string without the last character.
-	 */
-	private String cutBracket(String string) {
-		return string.substring(0, string.length() - 1);
-	}
-
-	/**
-	 * Save the result of the search into a JSONObject.
-	 * 
-	 * @return A special JSONObject which contain only one key. The value
-	 *         associated to this key is the result of the search.
-	 * @throws JSONException
-	 *             If there is a problem creating the JSONObject. (e.g. invalid
-	 *             syntax)
-	 */
-	private JSONObject putResultInJSONObj() throws JSONException {
-		JSONObject jo = new JSONObject();
-		jo.put("retValue", temp);
-		return jo;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java
deleted file mode 100644
index 8dd4458..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSource.java
+++ /dev/null
@@ -1,280 +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.connectors.twitter;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.twitter.hbc.core.endpoint.Location;
-import com.twitter.hbc.core.endpoint.StatusesFilterEndpoint;
-import com.twitter.hbc.httpclient.auth.Authentication;
-
-/**
- * 
- * An extension of {@link TwitterSource} by filter parameters. This extension
- * enables to filter the twitter stream by user defined parameters.
- */
-public class TwitterFilterSource extends TwitterSource {
-
-	private static final Logger LOG = LoggerFactory
-			.getLogger(TwitterFilterSource.class);
-
-	private static final long serialVersionUID = 1L;
-
-	private List<String> trackTerms = new LinkedList<String>();
-
-	private List<String> languages = new LinkedList<String>();
-
-	private List<Long> followings = new LinkedList<Long>();
-
-	private List<Location> locations = new LinkedList<Location>();
-
-	private Map<String, String> queryParameters = new HashMap<String, String>();
-
-	private Map<String, String> postParameters = new HashMap<String, String>();
-
-	public TwitterFilterSource(String authPath) {
-		super(authPath);
-	}
-
-	@Override
-	protected void initializeConnection() {
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Initializing Twitter Streaming API connection");
-		}
-		queue = new LinkedBlockingQueue<String>(queueSize);
-
-		StatusesFilterEndpoint endpoint = new StatusesFilterEndpoint();
-		configEndpoint(endpoint);
-		endpoint.stallWarnings(false);
-
-		Authentication auth = authenticate();
-
-		initializeClient(endpoint, auth);
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Twitter Streaming API connection established successfully");
-		}
-	}
-
-	/**
-	 * This function configures the streaming endpoint
-	 * 
-	 * @param endpoint
-	 *            The streaming endpoint
-	 */
-	private void configEndpoint(StatusesFilterEndpoint endpoint) {
-		if (!trackTerms.isEmpty()) {
-			endpoint.trackTerms(trackTerms);
-		}
-		if (!languages.isEmpty()) {
-			endpoint.languages(languages);
-		}
-		if (!followings.isEmpty()) {
-			endpoint.followings(followings);
-		}
-		if (!locations.isEmpty()) {
-			endpoint.locations(locations);
-		}
-		if (!queryParameters.isEmpty()) {
-			for (Entry<String, String> entry : queryParameters.entrySet()) {
-				endpoint.addQueryParameter(entry.getKey(), entry.getValue());
-			}
-		}
-		if (!postParameters.isEmpty()) {
-			for (Entry<String, String> entry : postParameters.entrySet()) {
-				endpoint.addPostParameter(entry.getKey(), entry.getValue());
-			}
-		}
-	}
-
-	/**
-	 * This function sets which term to track.
-	 * 
-	 * @param term
-	 *            The term to track.
-	 */
-	public void trackTerm(String term) {
-		this.trackTerms.add(term);
-	}
-
-	/**
-	 * This function sets which terms to track.
-	 * 
-	 * @param terms
-	 *            The terms to track.
-	 */
-	public void trackTerms(Collection<String> terms) {
-		this.trackTerms.addAll(terms);
-	}
-
-	/**
-	 * This function tells which terms are tracked.
-	 */
-	public List<String> getTrackTerms() {
-		return this.trackTerms;
-	}
-
-	/**
-	 * This function sets which language to filter.
-	 * 
-	 * @param language
-	 *            The language to filter.
-	 */
-	public void filterLanguage(String language) {
-		this.languages.add(language);
-	}
-
-	/**
-	 * This function sets which languages to filter.
-	 * 
-	 * @param languages
-	 *            The languages to filter.
-	 */
-	public void filterLanguages(Collection<String> languages) {
-		this.languages.addAll(languages);
-	}
-
-	/**
-	 * This function tells which languages are filtered.
-	 */
-	public List<String> getLanguages() {
-		return this.languages;
-	}
-
-	/**
-	 * This function sets which user to follow.
-	 * 
-	 * @param userID
-	 *            The ID of the user to follow.
-	 */
-	public void filterFollowings(Long userID) {
-		this.followings.add(userID);
-	}
-
-	/**
-	 * This function sets which users to follow.
-	 * 
-	 * @param userIDs
-	 *            The IDs of the users to follow.
-	 */
-	public void filterFollowings(Collection<Long> userIDs) {
-		this.followings.addAll(userIDs);
-	}
-
-	/**
-	 * This function tells which users are followed.
-	 */
-	public List<Long> getFollowings() {
-		return this.followings;
-	}
-
-	/**
-	 * This function sets which location to filter.
-	 * 
-	 * @param location
-	 *            The location to filter.
-	 */
-	public void filterLocation(Location location) {
-		this.locations.add(location);
-	}
-
-	/**
-	 * This function sets which locations to filter.
-	 * 
-	 * @param locations
-	 *            The locations to filter.
-	 */
-	public void filterLocations(Collection<Location> locations) {
-		this.locations.addAll(locations);
-	}
-
-	/**
-	 * This function tells which locations are filtered.
-	 */
-	public List<Location> getLocations() {
-		return this.locations;
-	}
-
-	/**
-	 * This function sets a query parameter.
-	 * 
-	 * @param parameter
-	 *            The name of the query parameter.
-	 * @param value
-	 *            The value of the query parameter.
-	 */
-	public void addQueryParameter(String parameter, String value) {
-		this.queryParameters.put(parameter, value);
-	}
-
-	/**
-	 * This function sets query parameters.
-	 * 
-	 * @param queryParameters
-	 *            The query parameters for the endpoint.
-	 */
-	public void addQueryParameters(Map<String, String> queryParameters) {
-		this.queryParameters.putAll(queryParameters);
-	}
-
-	/**
-	 * This function tells which query parameters are used by the endpoint.
-	 */
-	public Map<String, String> getQueryParameters() {
-		return this.queryParameters;
-	}
-
-	/**
-	 * This function sets a post parameter.
-	 * 
-	 * @param parameter
-	 *            The name of the post parameter.
-	 * @param value
-	 *            The value of the post parameter.
-	 */
-	public void addPostParameter(String parameter, String value) {
-		this.postParameters.put(parameter, value);
-	}
-
-	/**
-	 * This function sets post parameters.
-	 * 
-	 * @param postParameters
-	 *              The post parameters for the endpoint.
-	 */
-	public void addPostParameters(Map<String, String> postParameters) {
-		this.postParameters.putAll(postParameters);
-	}
-
-	/**
-	 * This function tells which post parameters are used by the endpoint.
-	 */
-	public Map<String, String> postParameters() {
-		return this.postParameters;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.java
deleted file mode 100644
index 43cb179..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterFilterSourceExample.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.connectors.twitter;
-
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.connectors.json.JSONParseFlatMap;
-import org.apache.flink.util.Collector;
-
-/**
- * This is an example how to use TwitterFilterSource. Before executing the
- * example you have to define the access keys of twitter.properties in the
- * resource folder. The access keys can be found in your twitter account.
- */
-public class TwitterFilterSourceExample {
-
-	/**
-	 * path to the twitter properties
-	 */
-	private static final String PATH_TO_AUTH_FILE = "/twitter.properties";
-
-	public static void main(String[] args) {
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment
-				.getExecutionEnvironment();
-
-		TwitterFilterSource twitterSource = new TwitterFilterSource(
-				TwitterFilterSourceExample.class.getResource(PATH_TO_AUTH_FILE)
-						.getFile());
-
-		twitterSource.trackTerm("obama");
-		twitterSource.filterLanguage("en");
-
-		DataStream<String> streamSource = env.addSource(twitterSource).flatMap(
-				new JSONParseFlatMap<String, String>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void flatMap(String s, Collector<String> c)
-							throws Exception {
-						c.collect(s);
-					}
-				});
-
-		streamSource.print();
-
-		try {
-			env.execute("Twitter Streaming Test");
-		} catch (Exception e) {
-			// TODO Auto-generated catch block
-			e.printStackTrace();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
deleted file mode 100644
index bad0f8c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
+++ /dev/null
@@ -1,233 +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.connectors.twitter;
-
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.util.Properties;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.twitter.hbc.ClientBuilder;
-import com.twitter.hbc.core.Constants;
-import com.twitter.hbc.core.endpoint.DefaultStreamingEndpoint;
-import com.twitter.hbc.core.endpoint.StatusesSampleEndpoint;
-import com.twitter.hbc.core.processor.StringDelimitedProcessor;
-import com.twitter.hbc.httpclient.BasicClient;
-import com.twitter.hbc.httpclient.auth.Authentication;
-import com.twitter.hbc.httpclient.auth.OAuth1;
-
-/**
- * Implementation of {@link SourceFunction} specialized to emit tweets from
- * Twitter. This is not a parallel source because the Twitter API only allows
- * two concurrent connections.
- */
-public class TwitterSource extends RichSourceFunction<String> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(TwitterSource.class);
-
-	private static final long serialVersionUID = 1L;
-	private String authPath;
-	protected transient BlockingQueue<String> queue;
-	protected int queueSize = 10000;
-	private transient BasicClient client;
-	private int waitSec = 5;
-
-	private int maxNumberOfTweets;
-	private int currentNumberOfTweets;
-
-	private transient volatile boolean isRunning;
-
-	/**
-	 * Create {@link TwitterSource} for streaming
-	 * 
-	 * @param authPath
-	 *            Location of the properties file containing the required
-	 *            authentication information.
-	 */
-	public TwitterSource(String authPath) {
-		this.authPath = authPath;
-		maxNumberOfTweets = -1;
-	}
-
-	/**
-	 * Create {@link TwitterSource} to collect finite number of tweets
-	 * 
-	 * @param authPath
-	 *            Location of the properties file containing the required
-	 *            authentication information.
-	 * @param numberOfTweets
-	 * 
-	 */
-	public TwitterSource(String authPath, int numberOfTweets) {
-		this.authPath = authPath;
-		this.maxNumberOfTweets = numberOfTweets;
-	}
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		initializeConnection();
-		currentNumberOfTweets = 0;
-		isRunning = true;
-	}
-
-	/**
-	 * Initialize Hosebird Client to be able to consume Twitter's Streaming API
-	 */
-	protected void initializeConnection() {
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Initializing Twitter Streaming API connection");
-		}
-
-		queue = new LinkedBlockingQueue<String>(queueSize);
-
-		StatusesSampleEndpoint endpoint = new StatusesSampleEndpoint();
-		endpoint.stallWarnings(false);
-
-		Authentication auth = authenticate();
-
-		initializeClient(endpoint, auth);
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Twitter Streaming API connection established successfully");
-		}
-	}
-
-	protected OAuth1 authenticate() {
-
-		Properties authenticationProperties = loadAuthenticationProperties();
-		
-		return new OAuth1(authenticationProperties.getProperty("consumerKey"),
-				authenticationProperties.getProperty("consumerSecret"),
-				authenticationProperties.getProperty("token"),
-				authenticationProperties.getProperty("secret"));
-	}
-
-	/**
-	 * Reads the given properties file for the authentication data.
-	 * 
-	 * @return the authentication data.
-	 */
-	private Properties loadAuthenticationProperties() {
-		
-		Properties properties = new Properties();
-		try {
-			InputStream input = new FileInputStream(authPath);
-			properties.load(input);
-			input.close();
-		} catch (Exception e) {
-			throw new RuntimeException("Cannot open .properties file: " + authPath, e);
-		}
-		return properties;
-	}
-
-	protected void initializeClient(DefaultStreamingEndpoint endpoint, Authentication auth) {
-
-		client = new ClientBuilder().name("twitterSourceClient").hosts(Constants.STREAM_HOST)
-				.endpoint(endpoint).authentication(auth)
-				.processor(new StringDelimitedProcessor(queue)).build();
-		
-		client.connect();
-	}
-
-	@Override
-	public void close() {
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Initiating connection close");
-		}
-
-		if (client != null) {
-			client.stop();
-		}
-
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Connection closed successfully");
-		}
-	}
-
-	/**
-	 * Get the size of the queue in which the tweets are contained temporarily.
-	 * 
-	 * @return the size of the queue in which the tweets are contained
-	 *         temporarily
-	 */
-	public int getQueueSize() {
-		return queueSize;
-	}
-
-	/**
-	 * Set the size of the queue in which the tweets are contained temporarily.
-	 * 
-	 * @param queueSize
-	 *            The desired value.
-	 */
-	public void setQueueSize(int queueSize) {
-		this.queueSize = queueSize;
-	}
-
-	/**
-	 * This function tells how long TwitterSource waits for the tweets.
-	 * 
-	 * @return Number of second.
-	 */
-	public int getWaitSec() {
-		return waitSec;
-	}
-
-	/**
-	 * This function sets how long TwitterSource should wait for the tweets.
-	 * 
-	 * @param waitSec
-	 *            The desired value.
-	 */
-	public void setWaitSec(int waitSec) {
-		this.waitSec = waitSec;
-	}
-
-	@Override
-	public void run(SourceContext<String> ctx) throws Exception {
-		while (isRunning) {
-			if (client.isDone()) {
-				if (LOG.isErrorEnabled()) {
-					LOG.error("Client connection closed unexpectedly: {}", client.getExitEvent()
-							.getMessage());
-				}
-				break;
-			}
-
-			ctx.collect(queue.take());
-
-			if (maxNumberOfTweets != -1 && currentNumberOfTweets >= maxNumberOfTweets) {
-				break;
-			}
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
deleted file mode 100644
index a80c32a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterStreaming.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.flink.streaming.connectors.twitter;
-
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.connectors.json.JSONParseFlatMap;
-import org.apache.flink.util.Collector;
-import org.apache.sling.commons.json.JSONException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TwitterStreaming {
-
-	private static final int PARALLELISM = 1;
-	private static final int SOURCE_PARALLELISM = 1;
-	private static final int NUMBEROFTWEETS = 100;
-
-	private static final Logger LOG = LoggerFactory.getLogger(TwitterStreaming.class);
-
-	public static class TwitterSink implements SinkFunction<Tuple5<Long, Integer, String, String, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void invoke(Tuple5<Long, Integer, String, String, String> tuple) {
-			System.out.println("ID: " + tuple.f0 + " int: " + tuple.f1 + " LANGUAGE: " + tuple.f2);
-			System.out.println("NAME: " + tuple.f4);
-			System.out.println("TEXT: " + tuple.f3);
-			System.out.println("");
-		}
-
-	}
-
-	public static class SelectDataFlatMap extends
-			JSONParseFlatMap<String, Tuple5<Long, Integer, String, String, String>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<Tuple5<Long, Integer, String, String, String>> out)
-				throws Exception {
-			try {
-				out.collect(new Tuple5<Long, Integer, String, String, String>(
-						getLong(value, "id"),
-						getInt(value, "entities.hashtags[0].indices[1]"),
-						getString(value, "lang"),
-						getString(value, "text"),
-						getString(value, "user.name")));
-			} catch (JSONException e) {
-				if (LOG.isErrorEnabled()) {
-					LOG.error("Field not found");
-				}
-			}
-		}
-	}
-
-	public static void main(String[] args) throws Exception {
-
-		String path = new String();
-
-		if (args != null && args.length == 1) {
-			path = args[0];
-		} else {
-			System.err.println("USAGE:\nTwitterStreaming <pathToPropertiesFile>");
-			return;
-		}
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment
-				.createLocalEnvironment(PARALLELISM);
-
-		DataStream<String> streamSource = env.addSource(new TwitterSource(path, NUMBEROFTWEETS))
-				.setParallelism(SOURCE_PARALLELISM);
-
-		DataStream<Tuple5<Long, Integer, String, String, String>> selectedDataStream = streamSource
-				.flatMap(new SelectDataFlatMap());
-
-		selectedDataStream.addSink(new TwitterSink());
-
-		env.execute();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
deleted file mode 100644
index b1fc92c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterTopology.java
+++ /dev/null
@@ -1,92 +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.connectors.twitter;
-
-import org.apache.flink.api.common.functions.MapFunction;
-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.connectors.json.JSONParseFlatMap;
-import org.apache.flink.util.Collector;
-import org.apache.sling.commons.json.JSONException;
-
-/**
-* This program demonstrate the use of TwitterSource.
-* Its aim is to count the frequency of the languages of tweets
-*/
-public class TwitterTopology {
-
-	private static final int NUMBEROFTWEETS = 100;
-
-	/**
-	 * FlatMapFunction to determine the language of tweets if possible
-	 */
-	public static class SelectLanguageFlatMap extends
-			JSONParseFlatMap<String, String> {
-
-		private static final long serialVersionUID = 1L;
-
-		/**
-		 * Select the language from the incoming JSON text
-		 */
-		@Override
-		public void flatMap(String value, Collector<String> out) throws Exception {
-			try{
-				out.collect(getString(value, "lang"));
-			}
-			catch (JSONException e){
-				out.collect("");
-			}
-		}
-
-	}
-
-	public static void main(String[] args) throws Exception {
-
-		String path = new String();
-
-		if (args != null && args.length == 1) {
-			path = args[0];
-		} else {
-			System.err.println("USAGE:\nTwitterLocal <pathToPropertiesFile>");
-			return;
-		}
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<String> streamSource = env.addSource(new TwitterSource(path, NUMBEROFTWEETS));
-
-
-		DataStream<Tuple2<String, Integer>> dataStream = streamSource
-				.flatMap(new SelectLanguageFlatMap())
-				.map(new MapFunction<String, Tuple2<String, Integer>>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public Tuple2<String, Integer> map(String value) throws Exception {
-						return new Tuple2<String, Integer>(value, 1);
-					}
-				})
-				.keyBy(0)
-				.sum(1);
-
-		dataStream.print();
-
-		env.execute();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties
deleted file mode 100644
index 1ca4143..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/main/resources/twitter.properties
+++ /dev/null
@@ -1,19 +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.
-
-secret=***
-consumerSecret=***
-token=***-***
-consumerKey=***

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
deleted file mode 100644
index b1d4115..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.json;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-import java.util.Collection;
-
-import org.apache.flink.streaming.connectors.json.JSONParser;
-import org.apache.sling.commons.json.JSONException;
-import org.apache.sling.commons.json.JSONObject;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-@RunWith(Parameterized.class)
-public class JSONParserTest {
-
-	private String jsonText;
-	private String searchedField;
-
-	public JSONParserTest(String text, String field) {
-		jsonText = text;
-		searchedField = field;
-	}
-
-	@Parameters
-	public static Collection<Object[]> initParameterList() {
-
-		Object[][] parameterList = new Object[][] { 
-				{ "{\"key\":\"value\"}", 							"key" },
-				{ "{\"key\":[\"value\"]}", 							"key[0]" },
-				{ "{\"key\":[{\"key\":\"value\"}]}", 				"key[0].key" },
-				{ "{\"key\":[{\"key\":[{\"key\":\"value\"}]}]}", 	"key[0].key[0].key"},
-				{ "{\"key\":[1,[{\"key\":\"value\"}]]}", 			"key[1][0].key" },
-				{ "{\"key\":[1,[[\"key\",2,\"value\"]]]}", 			"key[1][0][2]" },
-				{ "{\"key\":{\"key\":{\"otherKey\":\"wrongValue\",\"key\":\"value\"},\"otherKey\":\"wrongValue\"},\"otherKey\":\"wrongValue\"}" , "key.key.key"}
-				};
-
-		return Arrays.asList(parameterList);
-	}
-
-	@Test
-	public void test() {
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-			String expected = "{\"retValue\":\"value\"}";
-
-			assertTrue(expected.equals(jo.toString()));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
deleted file mode 100644
index 8851086..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/java/org/apache/flink/streaming/connectors/json/JSONParserTest2.java
+++ /dev/null
@@ -1,95 +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.connectors.json;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.flink.streaming.connectors.json.JSONParser;
-import org.apache.sling.commons.json.JSONException;
-import org.apache.sling.commons.json.JSONObject;
-import org.junit.Test;
-
-
-public class JSONParserTest2 {
-	
-	@Test
-	public void testGetBooleanFunction() {
-		String jsonText = "{\"key\":true}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertTrue(jo.getBoolean("retValue"));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-	
-	@Test
-	public void testGetDoubleFunction() {
-		double expected = 12345.12345;
-		String jsonText = "{\"key\":" + expected + "}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertEquals(expected,jo.getDouble("retValue"),0.000001);
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-	
-	@Test
-	public void testGetIntFunction() {
-		int expected = 15;
-		String jsonText = "{\"key\":" + expected + "}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertEquals(expected,jo.getInt("retValue"));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-
-	@Test
-	public void testGetLongFunction() {
-		long expected = 111111111111L;
-		String jsonText = "{\"key\":" + expected + "}";
-		String searchedField = "key";
-		try {
-			JSONParser parser = new JSONParser(jsonText);
-			JSONObject jo = parser.parse(searchedField);
-
-			assertEquals(expected,jo.getLong("retValue"));
-		} 
-		catch (JSONException e) {
-			fail();
-		}
-	}
-	
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties
deleted file mode 100644
index 9ede613..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-log4j.rootLogger=INFO, testlogger
-
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
-
-# suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml
deleted file mode 100644
index 45b3b92..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-twitter/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +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.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.streaming" level="WARN"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml
deleted file mode 100644
index 822ca26..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/pom.xml
+++ /dev/null
@@ -1,66 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-streaming-connectors-parent</artifactId>
-	<name>flink-streaming-connectors</name>
-
-	<packaging>pom</packaging>
-
-	<modules>
-		<module>flink-connector-flume</module>
-		<module>flink-connector-kafka</module>
-		<module>flink-connector-elasticsearch</module>
-		<module>flink-connector-rabbitmq</module>
-		<module>flink-connector-twitter</module>
-		<module>flink-connector-nifi</module>
-	</modules>
-
-	<!-- See main pom.xml for explanation of profiles -->
-	<profiles>
-		<profile>
-			<id>hadoop-2</id>
-			<activation>
-				<property>
-					<!-- Please do not remove the 'hadoop2' comment. See ./tools/generate_specific_pom.sh -->
-					<!--hadoop2--><name>!hadoop.profile</name>
-				</property>
-			</activation>
-			<modules>
-				<!-- Include the flink-fs-tests project only for HD2.
-				 	The HDFS minicluster interfaces changed between the two versions.
-				 -->
-				<module>flink-connector-filesystem</module>
-			</modules>
-		</profile>
-	</profiles>
-
-</project>


[11/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
new file mode 100644
index 0000000..65904d2
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
@@ -0,0 +1,284 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.fs;
+
+import com.google.common.collect.Sets;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.test.checkpointing.StreamFaultToleranceTestBase;
+import org.apache.flink.util.NetUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.Random;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link org.apache.flink.streaming.connectors.fs.RollingSink}.
+ *
+ * <p>
+ * This test only verifies the exactly once behaviour of the sink. Another test tests the
+ * rolling behaviour.
+ */
+public class RollingSinkFaultToleranceITCase extends StreamFaultToleranceTestBase {
+
+	final long NUM_STRINGS = 16_000;
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	private static MiniDFSCluster hdfsCluster;
+	private static org.apache.hadoop.fs.FileSystem dfs;
+
+	private static String outPath;
+
+
+
+	@BeforeClass
+	public static void createHDFS() throws IOException {
+		Configuration conf = new Configuration();
+
+		File dataDir = tempFolder.newFolder();
+
+		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+		hdfsCluster = builder.build();
+
+		dfs = hdfsCluster.getFileSystem();
+
+		outPath = "hdfs://"
+				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
+				+ "/string-non-rolling-out";
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		if (hdfsCluster != null) {
+			hdfsCluster.shutdown();
+		}
+	}
+
+
+	@Override
+	public void testProgram(StreamExecutionEnvironment env) {
+		assertTrue("Broken test setup", NUM_STRINGS % 40 == 0);
+
+		int PARALLELISM = 6;
+
+		env.enableCheckpointing(200);
+		env.setParallelism(PARALLELISM);
+		env.disableOperatorChaining();
+
+		DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)).startNewChain();
+
+		DataStream<String> mapped = stream
+				.map(new OnceFailingIdentityMapper(NUM_STRINGS));
+
+		RollingSink<String> sink = new RollingSink<String>(outPath)
+				.setBucketer(new NonRollingBucketer())
+				.setBatchSize(10000)
+				.setValidLengthPrefix("")
+				.setPendingPrefix("");
+
+		mapped.addSink(sink);
+
+	}
+
+	@Override
+	public void postSubmit() throws Exception {
+		// We read the files and verify that we have read all the strings. If a valid-length
+		// file exists we only read the file to that point. (This test should work with
+		// FileSystems that support truncate() and with others as well.)
+
+		Pattern messageRegex = Pattern.compile("message (\\d*)");
+
+		// Keep a set of the message IDs that we read. The size must equal the read count and
+		// the NUM_STRINGS. If numRead is bigger than the size of the set we have seen some
+		// elements twice.
+		Set<Integer> readNumbers = Sets.newHashSet();
+		int numRead = 0;
+
+		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(
+				outPath), true);
+
+		while (files.hasNext()) {
+			LocatedFileStatus file = files.next();
+
+			if (!file.getPath().toString().endsWith(".valid-length")) {
+				int validLength = (int) file.getLen();
+				if (dfs.exists(file.getPath().suffix(".valid-length"))) {
+					FSDataInputStream inStream = dfs.open(file.getPath().suffix(".valid-length"));
+					String validLengthString = inStream.readUTF();
+					validLength = Integer.parseInt(validLengthString);
+					System.out.println("VALID LENGTH: " + validLength);
+				}
+				FSDataInputStream inStream = dfs.open(file.getPath());
+				byte[] buffer = new byte[validLength];
+				inStream.readFully(0, buffer, 0, validLength);
+				inStream.close();
+
+				ByteArrayInputStream bais = new ByteArrayInputStream(buffer);
+
+				InputStreamReader inStreamReader = new InputStreamReader(bais);
+				BufferedReader br = new BufferedReader(inStreamReader);
+
+				String line = br.readLine();
+				while (line != null) {
+					Matcher matcher = messageRegex.matcher(line);
+					if (matcher.matches()) {
+						numRead++;
+						int messageId = Integer.parseInt(matcher.group(1));
+						readNumbers.add(messageId);
+					} else {
+						Assert.fail("Read line does not match expected pattern.");
+					}
+					line = br.readLine();
+				}
+				br.close();
+				inStreamReader.close();
+				bais.close();
+			}
+		}
+
+		// Verify that we read all strings (at-least-once)
+		Assert.assertEquals(NUM_STRINGS, readNumbers.size());
+
+		// Verify that we don't have duplicates (boom!, exactly-once)
+		Assert.assertEquals(NUM_STRINGS, numRead);
+	}
+
+	private static class OnceFailingIdentityMapper extends RichMapFunction<String, String> {
+		private static final long serialVersionUID = 1L;
+
+		private static volatile boolean hasFailed = false;
+
+		private final long numElements;
+
+		private long failurePos;
+		private long count;
+
+
+		OnceFailingIdentityMapper(long numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void open(org.apache.flink.configuration.Configuration parameters) throws IOException {
+			long failurePosMin = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
+			long failurePosMax = (long) (0.9 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
+
+			failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
+			count = 0;
+		}
+
+		@Override
+		public String map(String value) throws Exception {
+			count++;
+			if (!hasFailed && count >= failurePos) {
+				hasFailed = true;
+				throw new Exception("Test Failure");
+			}
+
+			return value;
+		}
+	}
+
+	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
+			implements CheckpointedAsynchronously<Integer> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final long numElements;
+
+		private int index;
+
+		private volatile boolean isRunning = true;
+
+
+		StringGeneratingSourceFunction(long numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<String> ctx) throws Exception {
+			final Object lockingObject = ctx.getCheckpointLock();
+
+			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
+
+			if (index == 0) {
+				index = getRuntimeContext().getIndexOfThisSubtask();
+			}
+
+			while (isRunning && index < numElements) {
+
+				Thread.sleep(1);
+				synchronized (lockingObject) {
+					ctx.collect("message " + index);
+					index += step;
+				}
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		private static String randomString(StringBuilder bld, Random rnd) {
+			final int len = rnd.nextInt(10) + 5;
+
+			for (int i = 0; i < len; i++) {
+				char next = (char) (rnd.nextInt(20000) + 33);
+				bld.append(next);
+			}
+
+			return bld.toString();
+		}
+
+		@Override
+		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+			return index;
+		}
+
+		@Override
+		public void restoreState(Integer state) {
+			index = state;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
new file mode 100644
index 0000000..9770f41
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
@@ -0,0 +1,506 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS 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.connectors.fs;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.taskmanager.MultiShotLatch;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.NetUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+/**
+ * Tests for {@link RollingSink}. These
+ * tests test the different output methods as well as the rolling feature using a manual clock
+ * that increases time in lockstep with element computation using latches.
+ *
+ * <p>
+ * This only tests the rolling behaviour of the sink. There is a separate ITCase that verifies
+ * exactly once behaviour.
+ */
+public class RollingSinkITCase extends StreamingMultipleProgramsTestBase {
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	private static MiniDFSCluster hdfsCluster;
+	private static org.apache.hadoop.fs.FileSystem dfs;
+	private static String hdfsURI;
+
+
+	@BeforeClass
+	public static void createHDFS() throws IOException {
+		Configuration conf = new Configuration();
+
+		File dataDir = tempFolder.newFolder();
+
+		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+		hdfsCluster = builder.build();
+
+		dfs = hdfsCluster.getFileSystem();
+
+		hdfsURI = "hdfs://"
+				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
+				+ "/";
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		hdfsCluster.shutdown();
+	}
+
+	/**
+	 * This tests {@link StringWriter} with
+	 * non-rolling output.
+	 */
+	@Test
+	public void testNonRollingStringWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/string-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+		RollingSink<String> sink = new RollingSink<String>(outPath)
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		source
+				.map(new MapFunction<Tuple2<Integer,String>, String>() {
+					private static final long serialVersionUID = 1L;
+					@Override
+					public String map(Tuple2<Integer, String> value) throws Exception {
+						return value.f1;
+					}
+				})
+				.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+
+		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			String line = br.readLine();
+			Assert.assertEquals("message #" + i, line);
+		}
+
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+
+		br = new BufferedReader(new InputStreamReader(inStream));
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			String line = br.readLine();
+			Assert.assertEquals("message #" + i, line);
+		}
+
+		inStream.close();
+	}
+
+	/**
+	 * This tests {@link SequenceFileWriter}
+	 * with non-rolling output and without compression.
+	 */
+	@Test
+	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/seq-no-comp-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
+				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
+			}
+		});
+
+
+		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
+				.setWriter(new SequenceFileWriter<IntWritable, Text>())
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		mapped.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+
+		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		IntWritable intWritable = new IntWritable();
+		Text txt = new Text();
+
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+
+		reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+	}
+
+	/**
+	 * This tests {@link SequenceFileWriter}
+	 * with non-rolling output but with compression.
+	 */
+	@Test
+	public void testNonRollingSequenceFileWithCompressionWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/seq-non-rolling-out";
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
+				.broadcast()
+				.filter(new OddEvenFilter());
+
+		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
+			private static final long serialVersionUID = 1L;
+
+			@Override
+			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
+				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
+			}
+		});
+
+
+		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
+				.setWriter(new SequenceFileWriter<IntWritable, Text>("Default", SequenceFile.CompressionType.BLOCK))
+				.setBucketer(new NonRollingBucketer())
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		mapped.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
+
+		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		IntWritable intWritable = new IntWritable();
+		Text txt = new Text();
+
+		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+
+		inStream = dfs.open(new Path(outPath + "/part-1-0"));
+
+		reader = new SequenceFile.Reader(inStream,
+				1000,
+				0,
+				100000,
+				new Configuration());
+
+		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
+			reader.next(intWritable, txt);
+			Assert.assertEquals(i, intWritable.get());
+			Assert.assertEquals("message #" + i, txt.toString());
+		}
+
+		reader.close();
+		inStream.close();
+	}
+
+	// we use this to synchronize the clock changes to elements being processed
+	final static MultiShotLatch latch1 = new MultiShotLatch();
+	final static MultiShotLatch latch2 = new MultiShotLatch();
+
+	/**
+	 * This uses {@link org.apache.flink.streaming.connectors.fs.DateTimeBucketer} to
+	 * produce rolling files. The clock of DateTimeBucketer is set to
+	 * {@link ModifyableClock} to keep the time in lockstep with the processing of elements using
+	 * latches.
+	 */
+	@Test
+	public void testDateTimeRollingStringWriter() throws Exception {
+		final int NUM_ELEMENTS = 20;
+		final int PARALLELISM = 2;
+		final String outPath = hdfsURI + "/rolling-out";
+		DateTimeBucketer.setClock(new ModifyableClock());
+		ModifyableClock.setCurrentTime(0);
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(PARALLELISM);
+
+
+
+		DataStream<Tuple2<Integer, String>> source = env.addSource(new WaitingTestSourceFunction(
+				NUM_ELEMENTS))
+				.broadcast();
+
+		// the parallel flatMap is chained to the sink, so when it has seen 5 elements it can
+		// fire the latch
+		DataStream<String> mapped = source
+				.flatMap(new RichFlatMapFunction<Tuple2<Integer, String>, String>() {
+					private static final long serialVersionUID = 1L;
+
+					int count = 0;
+					@Override
+					public void flatMap(Tuple2<Integer, String> value,
+							Collector<String> out) throws Exception {
+						out.collect(value.f1);
+						count++;
+						if (count >= 5) {
+							if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
+								latch1.trigger();
+							} else {
+								latch2.trigger();
+							}
+							count = 0;
+						}
+					}
+
+				});
+
+		RollingSink<String> sink = new RollingSink<String>(outPath)
+				.setBucketer(new DateTimeBucketer("ss"))
+				.setPartPrefix("part")
+				.setPendingPrefix("")
+				.setPendingSuffix("");
+
+		mapped.addSink(sink);
+
+		env.execute("RollingSink String Write Test");
+
+		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(outPath), true);
+
+		// we should have 8 rolling files, 4 time intervals and parallelism of 2
+		int numFiles = 0;
+		while (files.hasNext()) {
+			LocatedFileStatus file = files.next();
+			numFiles++;
+			if (file.getPath().toString().contains("rolling-out/00")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 0; i < 5; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/05")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 5; i < 10; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/10")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 10; i < 15; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else if (file.getPath().toString().contains("rolling-out/15")) {
+				FSDataInputStream inStream = dfs.open(file.getPath());
+
+				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
+
+				for (int i = 15; i < 20; i++) {
+					String line = br.readLine();
+					Assert.assertEquals("message #" + i, line);
+				}
+
+				inStream.close();
+			} else {
+				Assert.fail("File " + file + " does not match any expected roll pattern.");
+			}
+		}
+
+		Assert.assertEquals(8, numFiles);
+	}
+
+
+	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		private final int numElements;
+
+		public TestSourceFunction(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
+			for (int i = 0; i < numElements && running; i++) {
+				ctx.collect(Tuple2.of(i, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	/**
+	 * This waits on the two multi-shot latches. The latches are triggered in a parallel
+	 * flatMap inside the test topology.
+	 */
+	private static class WaitingTestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		private final int numElements;
+
+		public WaitingTestSourceFunction(int numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
+			for (int i = 0; i < numElements && running; i++) {
+				if (i % 5 == 0 && i > 0) {
+					// update the clock after "five seconds", so we get 20 seconds in total
+					// with 5 elements in each time window
+					latch1.await();
+					latch2.await();
+					ModifyableClock.setCurrentTime(i * 1000);
+				}
+				ctx.collect(Tuple2.of(i, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	public static class OddEvenFilter extends RichFilterFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public boolean filter(Tuple2<Integer, String> value) throws Exception {
+			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
+				return value.f0 % 2 == 0;
+			} else {
+				return value.f0 % 2 == 1;
+			}
+		}
+	}
+
+	public static class ModifyableClock implements Clock {
+
+		private static volatile long currentTime = 0;
+
+		public static void setCurrentTime(long currentTime) {
+			ModifyableClock.currentTime = currentTime;
+		}
+
+		@Override
+		public long currentTimeMillis() {
+			return currentTime;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..fe60d94
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties
@@ -0,0 +1,27 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+log4j.rootLogger=OFF, testlogger
+
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..45b3b92
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml
@@ -0,0 +1,30 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+    <logger name="org.apache.flink.streaming" level="WARN"/>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-flume/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-flume/pom.xml b/flink-streaming-connectors/flink-connector-flume/pom.xml
new file mode 100644
index 0000000..912a7e4
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-flume/pom.xml
@@ -0,0 +1,174 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-streaming-connectors-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-flume</artifactId>
+	<name>flink-connector-flume</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<flume-ng.version>1.5.0</flume-ng.version>
+	</properties>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flume</groupId>
+			<artifactId>flume-ng-core</artifactId>
+			<version>${flume-ng.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-log4j12</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>log4j</groupId>
+					<artifactId>log4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>commons-io</groupId>
+					<artifactId>commons-io</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>commons-codec</groupId>
+					<artifactId>commons-codec</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>commons-cli</groupId>
+					<artifactId>commons-cli</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>commons-lang</groupId>
+					<artifactId>commons-lang</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.avro</groupId>
+					<artifactId>avro</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-core-asl</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-mapper-asl</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.thoughtworks.paranamer</groupId>
+					<artifactId>paranamer</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.xerial.snappy</groupId>
+					<artifactId>snappy-java</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.tukaani</groupId>
+					<artifactId>xz</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.velocity</groupId>
+					<artifactId>velocity</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>commons-collections</groupId>
+					<artifactId>commons-collections</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty-util</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.mortbay.jetty</groupId>
+					<artifactId>jetty</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.google.code.gson</groupId>
+					<artifactId>gson</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.apache.thrift</groupId>
+					<artifactId>libthrift</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
+			<plugin>
+				<!-- Override artifactSet configuration to build fat-jar with all dependencies packed. -->
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<configuration>
+							<artifactSet>
+								<includes combine.children="append">
+									<!-- We include all dependencies that transitively depend on guava -->
+									<include>org.apache.flume:*</include>
+								</includes>
+							</artifactSet>
+							<transformers>
+								<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
+							</transformers>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
new file mode 100644
index 0000000..50f5770
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.flume;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.FlumeException;
+import org.apache.flume.api.RpcClient;
+import org.apache.flume.api.RpcClientFactory;
+import org.apache.flume.event.EventBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlumeSink<IN> extends RichSinkFunction<IN> {
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(FlumeSink.class);
+
+	private transient FlinkRpcClientFacade client;
+	boolean initDone = false;
+	String host;
+	int port;
+	SerializationSchema<IN, byte[]> schema;
+
+	public FlumeSink(String host, int port, SerializationSchema<IN, byte[]> schema) {
+		this.host = host;
+		this.port = port;
+		this.schema = schema;
+	}
+
+	/**
+	 * Receives tuples from the Apache Flink {@link DataStream} and forwards
+	 * them to Apache Flume.
+	 * 
+	 * @param value
+	 *            The tuple arriving from the datastream
+	 */
+	@Override
+	public void invoke(IN value) {
+
+		byte[] data = schema.serialize(value);
+		client.sendDataToFlume(data);
+
+	}
+
+	private class FlinkRpcClientFacade {
+		private RpcClient client;
+		private String hostname;
+		private int port;
+
+		/**
+		 * Initializes the connection to Apache Flume.
+		 * 
+		 * @param hostname
+		 *            The host
+		 * @param port
+		 *            The port.
+		 */
+		public void init(String hostname, int port) {
+			// Setup the RPC connection
+			this.hostname = hostname;
+			this.port = port;
+			int initCounter = 0;
+			while (true) {
+				if (initCounter >= 90) {
+					throw new RuntimeException("Cannot establish connection with" + port + " at "
+							+ host);
+				}
+				try {
+					this.client = RpcClientFactory.getDefaultInstance(hostname, port);
+				} catch (FlumeException e) {
+					// Wait one second if the connection failed before the next
+					// try
+					try {
+						Thread.sleep(1000);
+					} catch (InterruptedException e1) {
+						if (LOG.isErrorEnabled()) {
+							LOG.error("Interrupted while trying to connect {} at {}", port, host);
+						}
+					}
+				}
+				if (client != null) {
+					break;
+				}
+				initCounter++;
+			}
+			initDone = true;
+		}
+
+		/**
+		 * Sends byte arrays as {@link Event} series to Apache Flume.
+		 * 
+		 * @param data
+		 *            The byte array to send to Apache FLume
+		 */
+		public void sendDataToFlume(byte[] data) {
+			Event event = EventBuilder.withBody(data);
+
+			try {
+				client.append(event);
+
+			} catch (EventDeliveryException e) {
+				// clean up and recreate the client
+				client.close();
+				client = null;
+				client = RpcClientFactory.getDefaultInstance(hostname, port);
+			}
+		}
+
+	}
+
+	@Override
+	public void close() {
+		client.client.close();
+	}
+
+	@Override
+	public void open(Configuration config) {
+		client = new FlinkRpcClientFacade();
+		client.init(host, port);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
new file mode 100644
index 0000000..8fecd0f
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
@@ -0,0 +1,149 @@
+///*
+// * Licensed to the Apache Software Foundation (ASF) under one or more
+// * contributor license agreements.  See the NOTICE file distributed with
+// * this work for additional information regarding copyright ownership.
+// * The ASF licenses this file to You under the Apache License, Version 2.0
+// * (the "License"); you may not use this file except in compliance with
+// * the License.  You may obtain a copy of the License at
+// *
+// *    http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.flume;
+//
+//import java.util.List;
+//
+//import org.apache.flink.streaming.api.datastream.DataStream;
+//import org.apache.flink.streaming.api.functions.source.ConnectorSource;
+//import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+//import org.apache.flink.util.Collector;
+//import org.apache.flume.Context;
+//import org.apache.flume.channel.ChannelProcessor;
+//import org.apache.flume.source.AvroSource;
+//import org.apache.flume.source.avro.AvroFlumeEvent;
+//import org.apache.flume.source.avro.Status;
+//
+//public class FlumeSource<OUT> extends ConnectorSource<OUT> {
+//	private static final long serialVersionUID = 1L;
+//
+//	String host;
+//	String port;
+//	volatile boolean finished = false;
+//
+//	private volatile boolean isRunning = false;
+//
+//	FlumeSource(String host, int port, DeserializationSchema<OUT> deserializationSchema) {
+//		super(deserializationSchema);
+//		this.host = host;
+//		this.port = Integer.toString(port);
+//	}
+//
+//	public class MyAvroSource extends AvroSource {
+//		Collector<OUT> output;
+//
+//		/**
+//		 * Sends the AvroFlumeEvent from it's argument list to the Apache Flink
+//		 * {@link DataStream}.
+//		 *
+//		 * @param avroEvent
+//		 *            The event that should be sent to the dataStream
+//		 * @return A {@link Status}.OK message if sending the event was
+//		 *         successful.
+//		 */
+//		@Override
+//		public Status append(AvroFlumeEvent avroEvent) {
+//			collect(avroEvent);
+//			return Status.OK;
+//		}
+//
+//		/**
+//		 * Sends the AvroFlumeEvents from it's argument list to the Apache Flink
+//		 * {@link DataStream}.
+//		 *
+//		 * @param events
+//		 *            The events that is sent to the dataStream
+//		 * @return A Status.OK message if sending the events was successful.
+//		 */
+//		@Override
+//		public Status appendBatch(List<AvroFlumeEvent> events) {
+//			for (AvroFlumeEvent avroEvent : events) {
+//				collect(avroEvent);
+//			}
+//
+//			return Status.OK;
+//		}
+//
+//		/**
+//		 * Deserializes the AvroFlumeEvent before sending it to the Apache Flink
+//		 * {@link DataStream}.
+//		 *
+//		 * @param avroEvent
+//		 *            The event that is sent to the dataStream
+//		 */
+//		private void collect(AvroFlumeEvent avroEvent) {
+//			byte[] b = avroEvent.getBody().array();
+//			OUT out = FlumeSource.this.schema.deserialize(b);
+//
+//			if (schema.isEndOfStream(out)) {
+//				FlumeSource.this.finished = true;
+//				this.stop();
+//				FlumeSource.this.notifyAll();
+//			} else {
+//				output.collect(out);
+//			}
+//
+//		}
+//
+//	}
+//
+//	MyAvroSource avroSource;
+//
+//	/**
+//	 * Configures the AvroSource. Also sets the output so the application can
+//	 * use it from outside of the invoke function.
+//	 *
+//	 * @param output
+//	 *            The output used in the invoke function
+//	 */
+//	public void configureAvroSource(Collector<OUT> output) {
+//
+//		avroSource = new MyAvroSource();
+//		avroSource.output = output;
+//		Context context = new Context();
+//		context.put("port", port);
+//		context.put("bind", host);
+//		avroSource.configure(context);
+//		// An instance of a ChannelProcessor is required for configuring the
+//		// avroSource although it will not be used in this case.
+//		ChannelProcessor cp = new ChannelProcessor(null);
+//		avroSource.setChannelProcessor(cp);
+//	}
+//
+//	/**
+//	 * Configures the AvroSource and runs until the user calls a close function.
+//	 *
+//	 * @param output
+//	 *            The Collector for sending data to the datastream
+//	 */
+//	@Override
+//	public void run(Collector<OUT> output) throws Exception {
+//		isRunning = true;
+//		configureAvroSource(output);
+//		avroSource.start();
+//		while (!finished && isRunning) {
+//			this.wait();
+//		}
+//	}
+//
+//	@Override
+//	public void cancel() {
+//		isRunning = false;
+//	}
+//
+//}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
new file mode 100644
index 0000000..45da6eb
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
@@ -0,0 +1,49 @@
+///*
+// * Licensed to the Apache Software Foundation (ASF) under one or more
+// * contributor license agreements.  See the NOTICE file distributed with
+// * this work for additional information regarding copyright ownership.
+// * The ASF licenses this file to You under the Apache License, Version 2.0
+// * (the "License"); you may not use this file except in compliance with
+// * the License.  You may obtain a copy of the License at
+// *
+// *    http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.flink.streaming.connectors.flume;
+//
+//import org.apache.flink.streaming.api.datastream.DataStream;
+//import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+//import org.apache.flink.streaming.util.serialization.SerializationSchema;
+//import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+//
+//public class FlumeTopology {
+//
+//	public static void main(String[] args) throws Exception {
+//
+//		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+//
+//		@SuppressWarnings("unused")
+//		DataStream<String> inputStream1 = env.addSource(
+//				new FlumeSource<String>("localhost", 41414, new SimpleStringSchema())).addSink(
+//				new FlumeSink<String>("localhost", 42424, new StringToByteSerializer()));
+//
+//		env.execute();
+//	}
+//
+//	public static class StringToByteSerializer implements SerializationSchema<String, byte[]> {
+//
+//		private static final long serialVersionUID = 1L;
+//
+//		@Override
+//		public byte[] serialize(String element) {
+//			return element.getBytes();
+//		}
+//	}
+//
+//}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/pom.xml b/flink-streaming-connectors/flink-connector-kafka/pom.xml
new file mode 100644
index 0000000..57a9a56
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/pom.xml
@@ -0,0 +1,130 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-streaming-connectors-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-kafka</artifactId>
+	<name>flink-connector-kafka</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<kafka.version>0.8.2.0</kafka.version>
+	</properties>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.kafka</groupId>
+			<artifactId>kafka_${scala.binary.version}</artifactId>
+			<version>${kafka.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>com.sun.jmx</groupId>
+					<artifactId>jmxri</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.sun.jdmk</groupId>
+					<artifactId>jmxtools</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>log4j</groupId>
+					<artifactId>log4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-simple</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>net.sf.jopt-simple</groupId>
+					<artifactId>jopt-simple</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.scala-lang</groupId>
+					<artifactId>scala-reflect</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.scala-lang</groupId>
+					<artifactId>scala-compiler</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.yammer.metrics</groupId>
+					<artifactId>metrics-annotation</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.xerial.snappy</groupId>
+					<artifactId>snappy-java</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<version>${curator.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-failsafe-plugin</artifactId>
+				<configuration>
+					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
+					<forkCount>1</forkCount>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+	
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
new file mode 100644
index 0000000..8066b3c
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
@@ -0,0 +1,689 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+import kafka.cluster.Broker;
+import kafka.common.ErrorMapping;
+import kafka.javaapi.PartitionMetadata;
+import kafka.javaapi.TopicMetadata;
+import kafka.javaapi.TopicMetadataRequest;
+import kafka.javaapi.consumer.SimpleConsumer;
+import org.apache.commons.collections.map.LinkedMap;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.connectors.kafka.internals.Fetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.OffsetHandler;
+import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import org.apache.flink.util.NetUtils;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
+ * Apache Kafka. The consumer can run in multiple parallel instances, each of which will pull
+ * data from one or more Kafka partitions. 
+ * 
+ * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
+ * during a failure, and that the computation processes elements "exactly once". 
+ * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
+ * 
+ * <p>To support a variety of Kafka brokers, protocol versions, and offset committing approaches,
+ * the Flink Kafka Consumer can be parametrized with a <i>fetcher</i> and an <i>offset handler</i>.</p>
+ *
+ * <h1>Fetcher</h1>
+ * 
+ * <p>The fetcher is responsible to pull data from Kafka. Because Kafka has undergone a change in
+ * protocols and APIs, there are currently two fetchers available:</p>
+ * 
+ * <ul>
+ *     <li>{@link FetcherType#NEW_HIGH_LEVEL}: A fetcher based on the new Kafka consumer API.
+ *         This fetcher is generally more robust, but works only with later versions of
+ *         Kafka (> 0.8.2).</li>
+ *         
+ *     <li>{@link FetcherType#LEGACY_LOW_LEVEL}: A fetcher based on the old low-level consumer API.
+ *         This fetcher is works also with older versions of Kafka (0.8.1). The fetcher interprets
+ *         the old Kafka consumer properties, like:
+ *         <ul>
+ *             <li>socket.timeout.ms</li>
+ *             <li>socket.receive.buffer.bytes</li>
+ *             <li>fetch.message.max.bytes</li>
+ *             <li>auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)</li>
+ *             <li>fetch.wait.max.ms</li>
+ *         </ul>
+ *     </li>
+ * </ul>
+ * 
+ * <h1>Offset handler</h1>
+ * 
+ * <p>Offsets whose records have been read and are checkpointed will be committed back to Kafka / ZooKeeper
+ * by the offset handler. In addition, the offset handler finds the point where the source initially
+ * starts reading from the stream, when the streaming job is started.</p>
+ * 
+ * <p>Currently, the source offers two different offset handlers exist:</p>
+ * <ul>
+ *     <li>{@link OffsetStore#KAFKA}: Use this offset handler when the Kafka brokers are managing the offsets,
+ *         and hence offsets need to be committed the Kafka brokers, rather than to ZooKeeper.
+ *         Note that this offset handler works only on new versions of Kafka (0.8.2.x +) and
+ *         with the {@link FetcherType#NEW_HIGH_LEVEL} fetcher.</li>
+ *         
+ *     <li>{@link OffsetStore#FLINK_ZOOKEEPER}: Use this offset handler when the offsets are managed
+ *         by ZooKeeper, as in older versions of Kafka (0.8.1.x)</li>
+ * </ul>
+ * 
+ * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
+ * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
+ * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
+ * has consumed a topic.</p>
+ * 
+ * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
+ * is constructed. That means that the client that submits the program needs to be able to
+ * reach the Kafka brokers or ZooKeeper.</p>
+ */
+public class FlinkKafkaConsumer<T> extends RichParallelSourceFunction<T>
+		implements CheckpointNotifier, CheckpointedAsynchronously<long[]>, ResultTypeQueryable<T> {
+
+	/**
+	 * The offset store defines how acknowledged offsets are committed back to Kafka. Different
+	 * options include letting Flink periodically commit to ZooKeeper, or letting Kafka manage the
+	 * offsets (new Kafka versions only).
+	 */
+	public enum OffsetStore {
+
+		/**
+		 * Let Flink manage the offsets. Flink will periodically commit them to Zookeeper (usually after
+		 * successful checkpoints), in the same structure as Kafka 0.8.2.x
+		 * 
+		 * <p>Use this mode when using the source with Kafka 0.8.1.x brokers.</p>
+		 */
+		FLINK_ZOOKEEPER,
+
+		/**
+		 * Use the mechanisms in Kafka to commit offsets. Depending on the Kafka configuration, different
+		 * mechanism will be used (broker coordinator, zookeeper)
+		 */ 
+		KAFKA
+	}
+
+	/**
+	 * The fetcher type defines which code paths to use to pull data from teh Kafka broker.
+	 */
+	public enum FetcherType {
+
+		/**
+		 * The legacy fetcher uses Kafka's old low-level consumer API.
+		 * 
+		 * <p>Use this fetcher for Kafka 0.8.1 brokers.</p>
+		 */
+		LEGACY_LOW_LEVEL,
+
+		/**
+		 * This fetcher uses a backport of the new consumer API to pull data from the Kafka broker.
+		 * It is the fetcher that will be maintained in the future, and it already 
+		 * handles certain failure cases with less overhead than the legacy fetcher.
+		 * 
+		 * <p>This fetcher works only Kafka 0.8.2 and 0.8.3 (and future versions).</p>
+		 */
+		NEW_HIGH_LEVEL
+	}
+	
+	// ------------------------------------------------------------------------
+	
+	private static final long serialVersionUID = -6272159445203409112L;
+	
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer.class);
+
+	/** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid),
+	 * and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */
+	public static final long OFFSET_NOT_SET = -915623761776L;
+
+	/** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */
+	public static final int MAX_NUM_PENDING_CHECKPOINTS = 100;
+
+	/** Configuration key for the number of retries for getting the partition info */
+	public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry";
+
+	/** Default number of retries for getting the partition info. One retry means going through the full list of brokers */
+	public static final int DEFAULT_GET_PARTITIONS_RETRIES = 3;
+
+	
+	
+	// ------  Configuration of the Consumer -------
+	
+	/** The offset store where this consumer commits safe offsets */
+	private final OffsetStore offsetStore;
+
+	/** The type of fetcher to be used to pull data from Kafka */
+	private final FetcherType fetcherType;
+	
+	/** name of the topic consumed by this source */
+	private final String topic;
+	
+	/** The properties to parametrize the Kafka consumer and ZooKeeper client */ 
+	private final Properties props;
+	
+	/** The ids of the partitions that are read by this consumer */
+	private final int[] partitions;
+	
+	/** The schema to convert between Kafka#s byte messages, and Flink's objects */
+	private final DeserializationSchema<T> valueDeserializer;
+
+	// ------  Runtime State  -------
+
+	/** Data for pending but uncommitted checkpoints */
+	private final LinkedMap pendingCheckpoints = new LinkedMap();
+	
+	/** The fetcher used to pull data from the Kafka brokers */
+	private transient Fetcher fetcher;
+	
+	/** The committer that persists the committed offsets */
+	private transient OffsetHandler offsetHandler;
+	
+	/** The partitions actually handled by this consumer */
+	private transient List<TopicPartition> subscribedPartitions;
+
+	/** The offsets of the last returned elements */
+	private transient long[] lastOffsets;
+
+	/** The latest offsets that have been committed to Kafka or ZooKeeper. These are never
+	 * newer then the last offsets (Flink's internal view is fresher) */
+	private transient long[] commitedOffsets;
+	
+	/** The offsets to restore to, if the consumer restores state from a checkpoint */
+	private transient long[] restoreToOffset;
+	
+	private volatile boolean running = true;
+	
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler.
+	 * 
+	 * <p>To determine which kink of fetcher and offset handler to use, please refer to the docs
+	 * at the beginnign of this class.</p>
+	 * 
+	 * @param topic 
+	 *           The Kafka topic to read from.
+	 * @param valueDeserializer
+	 *           The deserializer to turn raw byte messages into Java/Scala objects.
+	 * @param props
+	 *           The properties that are used to configure both the fetcher and the offset handler.
+	 * @param offsetStore
+	 *           The type of offset store to use (Kafka / ZooKeeper)
+	 * @param fetcherType
+	 *           The type of fetcher to use (new high-level API, old low-level API).
+	 */
+	public FlinkKafkaConsumer(String topic, DeserializationSchema<T> valueDeserializer, Properties props, 
+								OffsetStore offsetStore, FetcherType fetcherType) {
+		this.offsetStore = checkNotNull(offsetStore);
+		this.fetcherType = checkNotNull(fetcherType);
+
+		if(fetcherType == FetcherType.NEW_HIGH_LEVEL) {
+			throw new UnsupportedOperationException("The fetcher for Kafka 0.8.3 is not yet " +
+					"supported in Flink");
+		}
+		if (offsetStore == OffsetStore.KAFKA && fetcherType == FetcherType.LEGACY_LOW_LEVEL) {
+			throw new IllegalArgumentException(
+					"The Kafka offset handler cannot be used together with the old low-level fetcher.");
+		}
+		
+		this.topic = checkNotNull(topic, "topic");
+		this.props = checkNotNull(props, "props");
+		this.valueDeserializer = checkNotNull(valueDeserializer, "valueDeserializer");
+
+		// validate the zookeeper properties
+		if (offsetStore == OffsetStore.FLINK_ZOOKEEPER) {
+			validateZooKeeperConfig(props);
+		}
+		
+		// Connect to a broker to get the partitions
+		List<PartitionInfo> partitionInfos = getPartitionsForTopic(topic, props);
+
+		// get initial partitions list. The order of the partitions is important for consistent 
+		// partition id assignment in restart cases.
+		this.partitions = new int[partitionInfos.size()];
+		for (int i = 0; i < partitionInfos.size(); i++) {
+			partitions[i] = partitionInfos.get(i).partition();
+			
+			if (partitions[i] >= partitions.length) {
+				throw new RuntimeException("Kafka partition numbers are sparse");
+			}
+		}
+		LOG.info("Topic {} has {} partitions", topic, partitions.length);
+
+		// make sure that we take care of the committing
+		props.setProperty("enable.auto.commit", "false");
+	}
+
+	// ------------------------------------------------------------------------
+	//  Source life cycle
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+		
+		final int numConsumers = getRuntimeContext().getNumberOfParallelSubtasks();
+		final int thisComsumerIndex = getRuntimeContext().getIndexOfThisSubtask();
+		
+		// pick which partitions we work on
+		subscribedPartitions = assignPartitions(this.partitions, this.topic, numConsumers, thisComsumerIndex);
+		
+		if (LOG.isInfoEnabled()) {
+			LOG.info("Kafka consumer {} will read partitions {} out of partitions {}",
+					thisComsumerIndex, subscribedPartitions, Arrays.toString(partitions));
+		}
+
+		// we leave the fetcher as null, if we have no partitions
+		if (subscribedPartitions.isEmpty()) {
+			LOG.info("Kafka consumer {} has no partitions (empty source)", thisComsumerIndex);
+			return;
+		}
+		
+		// create fetcher
+		switch (fetcherType){
+			case NEW_HIGH_LEVEL:
+				throw new UnsupportedOperationException("Currently unsupported");
+			case LEGACY_LOW_LEVEL:
+				fetcher = new LegacyFetcher(topic, props, getRuntimeContext().getTaskName());
+				break;
+			default:
+				throw new RuntimeException("Requested unknown fetcher " + fetcher);
+		}
+		fetcher.setPartitionsToRead(subscribedPartitions);
+
+		// offset handling
+		switch (offsetStore){
+			case FLINK_ZOOKEEPER:
+				offsetHandler = new ZookeeperOffsetHandler(props);
+				break;
+			case KAFKA:
+				throw new Exception("Kafka offset handler cannot work with legacy fetcher");
+			default:
+				throw new RuntimeException("Requested unknown offset store " + offsetStore);
+		}
+		
+		// set up operator state
+		lastOffsets = new long[partitions.length];
+		commitedOffsets = new long[partitions.length];
+		
+		Arrays.fill(lastOffsets, OFFSET_NOT_SET);
+		Arrays.fill(commitedOffsets, OFFSET_NOT_SET);
+		
+		// seek to last known pos, from restore request
+		if (restoreToOffset != null) {
+			if (LOG.isInfoEnabled()) {
+				LOG.info("Consumer {} found offsets from previous checkpoint: {}",
+						thisComsumerIndex,  Arrays.toString(restoreToOffset));
+			}
+			
+			for (int i = 0; i < restoreToOffset.length; i++) {
+				long restoredOffset = restoreToOffset[i];
+				if (restoredOffset != OFFSET_NOT_SET) {
+					// if this fails because we are not subscribed to the topic, then the
+					// partition assignment is not deterministic!
+					
+					// we set the offset +1 here, because seek() is accepting the next offset to read,
+					// but the restore offset is the last read offset
+					fetcher.seek(new TopicPartition(topic, i), restoredOffset + 1);
+					lastOffsets[i] = restoredOffset;
+				}
+			}
+		}
+		else {
+			// no restore request. Let the offset handler take care of the initial offset seeking
+			offsetHandler.seekFetcherToInitialOffsets(subscribedPartitions, fetcher);
+		}
+	}
+
+	@Override
+	public void run(SourceContext<T> sourceContext) throws Exception {
+		if (fetcher != null) {
+			fetcher.run(sourceContext, valueDeserializer, lastOffsets);
+		}
+		else {
+			// this source never completes
+			final Object waitLock = new Object();
+			while (running) {
+				// wait until we are canceled
+				try {
+					//noinspection SynchronizationOnLocalVariableOrMethodParameter
+					synchronized (waitLock) {
+						waitLock.wait();
+					}
+				}
+				catch (InterruptedException e) {
+					// do nothing, check our "running" status
+				}
+			}
+		}
+		
+		// close the context after the work was done. this can actually only
+		// happen when the fetcher decides to stop fetching
+		sourceContext.close();
+	}
+
+	@Override
+	public void cancel() {
+		// set ourselves as not running
+		running = false;
+		
+		// close the fetcher to interrupt any work
+		Fetcher fetcher = this.fetcher;
+		this.fetcher = null;
+		if (fetcher != null) {
+			try {
+				fetcher.close();
+			}
+			catch (IOException e) {
+				LOG.warn("Error while closing Kafka connector data fetcher", e);
+			}
+		}
+		
+		OffsetHandler offsetHandler = this.offsetHandler;
+		this.offsetHandler = null;
+		if (offsetHandler != null) {
+			try {
+				offsetHandler.close();
+			}
+			catch (IOException e) {
+				LOG.warn("Error while closing Kafka connector offset handler", e);
+			}
+		}
+	}
+
+	@Override
+	public void close() throws Exception {
+		cancel();
+		super.close();
+	}
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return valueDeserializer.getProducedType();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Checkpoint and restore
+	// ------------------------------------------------------------------------
+
+	@Override
+	public long[] snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		if (lastOffsets == null) {
+			LOG.debug("snapshotState() requested on not yet opened source; returning null.");
+			return null;
+		}
+		if (!running) {
+			LOG.debug("snapshotState() called on closed source");
+			return null;
+		}
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}",
+					Arrays.toString(lastOffsets), checkpointId, checkpointTimestamp);
+		}
+
+		long[] currentOffsets = Arrays.copyOf(lastOffsets, lastOffsets.length);
+
+		// the map cannot be asynchronously updated, because only one checkpoint call can happen
+		// on this function at a time: either snapshotState() or notifyCheckpointComplete()
+		pendingCheckpoints.put(checkpointId, currentOffsets);
+			
+		while (pendingCheckpoints.size() > MAX_NUM_PENDING_CHECKPOINTS) {
+			pendingCheckpoints.remove(0);
+		}
+
+		return currentOffsets;
+	}
+
+	@Override
+	public void restoreState(long[] restoredOffsets) {
+		restoreToOffset = restoredOffsets;
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		if (fetcher == null) {
+			LOG.debug("notifyCheckpointComplete() called on uninitialized source");
+			return;
+		}
+		if (!running) {
+			LOG.debug("notifyCheckpointComplete() called on closed source");
+			return;
+		}
+		
+		// only one commit operation must be in progress
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Committing offsets externally for checkpoint {}", checkpointId);
+		}
+
+		try {
+			long[] checkpointOffsets;
+	
+			// the map may be asynchronously updates when snapshotting state, so we synchronize
+			synchronized (pendingCheckpoints) {
+				final int posInMap = pendingCheckpoints.indexOf(checkpointId);
+				if (posInMap == -1) {
+					LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
+					return;
+				}
+	
+				checkpointOffsets = (long[]) pendingCheckpoints.remove(posInMap);
+				
+				// remove older checkpoints in map
+				for (int i = 0; i < posInMap; i++) {
+					pendingCheckpoints.remove(0);
+				}
+			}
+	
+			if (LOG.isInfoEnabled()) {
+				LOG.info("Committing offsets {} to offset store: {}", Arrays.toString(checkpointOffsets), offsetStore);
+			}
+	
+			// build the map of (topic,partition) -> committed offset
+			Map<TopicPartition, Long> offsetsToCommit = new HashMap<>();
+			for (TopicPartition tp : subscribedPartitions) {
+				
+				int partition = tp.partition();
+				long offset = checkpointOffsets[partition];
+				long lastCommitted = commitedOffsets[partition];
+				
+				if (offset != OFFSET_NOT_SET) {
+					if (offset > lastCommitted) {
+						offsetsToCommit.put(tp, offset);
+						LOG.debug("Committing offset {} for partition {}", offset, partition);
+					}
+					else {
+						LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, partition);
+					}
+				}
+			}
+			
+			offsetHandler.commit(offsetsToCommit);
+		}
+		catch (Exception e) {
+			if (running) {
+				throw e;
+			}
+			// else ignore exception if we are no longer running
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Miscellaneous utilities 
+	// ------------------------------------------------------------------------
+
+	protected static List<TopicPartition> assignPartitions(int[] partitions, String topicName,
+															int numConsumers, int consumerIndex) {
+		checkArgument(numConsumers > 0);
+		checkArgument(consumerIndex < numConsumers);
+		
+		List<TopicPartition> partitionsToSub = new ArrayList<>();
+
+		for (int i = 0; i < partitions.length; i++) {
+			if (i % numConsumers == consumerIndex) {
+				partitionsToSub.add(new TopicPartition(topicName, partitions[i]));
+			}
+		}
+		return partitionsToSub;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Kafka / ZooKeeper communication utilities
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Send request to Kafka to get partitions for topic.
+	 * 
+	 * @param topic The name of the topic.
+	 * @param properties The properties for the Kafka Consumer that is used to query the partitions for the topic. 
+	 */
+	public static List<PartitionInfo> getPartitionsForTopic(final String topic, final Properties properties) {
+		String seedBrokersConfString = properties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
+		final int numRetries = Integer.valueOf(properties.getProperty(GET_PARTITIONS_RETRIES_KEY, Integer.toString(DEFAULT_GET_PARTITIONS_RETRIES)));
+
+		checkNotNull(seedBrokersConfString, "Configuration property " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + " not set");
+		String[] seedBrokers = seedBrokersConfString.split(",");
+		List<PartitionInfo> partitions = new ArrayList<>();
+
+		Random rnd = new Random();
+		retryLoop: for(int retry = 0; retry < numRetries; retry++) {
+			// we pick a seed broker randomly to avoid overloading the first broker with all the requests when the
+			// parallel source instances start. Still, we try all available brokers.
+			int index = rnd.nextInt(seedBrokers.length);
+			brokersLoop: for (int arrIdx = 0; arrIdx < seedBrokers.length; arrIdx++) {
+				String seedBroker = seedBrokers[index];
+				LOG.info("Trying to get topic metadata from broker {} in try {}/{}", seedBroker, retry, numRetries);
+				if (++index == seedBrokers.length) {
+					index = 0;
+				}
+
+				URL brokerUrl = NetUtils.getCorrectHostnamePort(seedBroker);
+				SimpleConsumer consumer = null;
+				try {
+					final String clientId = "flink-kafka-consumer-partition-lookup";
+					final int soTimeout = Integer.valueOf(properties.getProperty("socket.timeout.ms", "30000"));
+					final int bufferSize = Integer.valueOf(properties.getProperty("socket.receive.buffer.bytes", "65536"));
+					consumer = new SimpleConsumer(brokerUrl.getHost(), brokerUrl.getPort(), soTimeout, bufferSize, clientId);
+
+					List<String> topics = Collections.singletonList(topic);
+					TopicMetadataRequest req = new TopicMetadataRequest(topics);
+					kafka.javaapi.TopicMetadataResponse resp = consumer.send(req);
+
+					List<TopicMetadata> metaData = resp.topicsMetadata();
+
+					// clear in case we have an incomplete list from previous tries
+					partitions.clear();
+					for (TopicMetadata item : metaData) {
+						if (item.errorCode() != ErrorMapping.NoError()) {
+							if (item.errorCode() == ErrorMapping.InvalidTopicCode() || item.errorCode() == ErrorMapping.UnknownTopicOrPartitionCode()) {
+								// fail hard if topic is unknown
+								throw new RuntimeException("Requested partitions for unknown topic", ErrorMapping.exceptionFor(item.errorCode()));
+							}
+							// warn and try more brokers
+							LOG.warn("Error while getting metadata from broker " + seedBroker + " to find partitions for " + topic,
+									ErrorMapping.exceptionFor(item.errorCode()));
+							continue brokersLoop;
+						}
+						if (!item.topic().equals(topic)) {
+							LOG.warn("Received metadata from topic " + item.topic() + " even though it was not requested. Skipping ...");
+							continue brokersLoop;
+						}
+						for (PartitionMetadata part : item.partitionsMetadata()) {
+							Node leader = brokerToNode(part.leader());
+							Node[] replicas = new Node[part.replicas().size()];
+							for (int i = 0; i < part.replicas().size(); i++) {
+								replicas[i] = brokerToNode(part.replicas().get(i));
+							}
+
+							Node[] ISRs = new Node[part.isr().size()];
+							for (int i = 0; i < part.isr().size(); i++) {
+								ISRs[i] = brokerToNode(part.isr().get(i));
+							}
+							PartitionInfo pInfo = new PartitionInfo(topic, part.partitionId(), leader, replicas, ISRs);
+							partitions.add(pInfo);
+						}
+					}
+					break retryLoop; // leave the loop through the brokers
+				} catch (Exception e) {
+					LOG.warn("Error communicating with broker " + seedBroker + " to find partitions for " + topic, e);
+				} finally {
+					if (consumer != null) {
+						consumer.close();
+					}
+				}
+			} // brokers loop
+		} // retries loop
+		return partitions;
+	}
+
+	private static Node brokerToNode(Broker broker) {
+		return new Node(broker.id(), broker.host(), broker.port());
+	}
+	
+	protected static void validateZooKeeperConfig(Properties props) {
+		if (props.getProperty("zookeeper.connect") == null) {
+			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set in the properties");
+		}
+		if (props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) {
+			throw new IllegalArgumentException("Required property '" + ConsumerConfig.GROUP_ID_CONFIG
+					+ "' has not been set in the properties");
+		}
+		
+		try {
+			//noinspection ResultOfMethodCallIgnored
+			Integer.parseInt(props.getProperty("zookeeper.session.timeout.ms", "0"));
+		}
+		catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Property 'zookeeper.session.timeout.ms' is not a valid integer");
+		}
+		
+		try {
+			//noinspection ResultOfMethodCallIgnored
+			Integer.parseInt(props.getProperty("zookeeper.connection.timeout.ms", "0"));
+		}
+		catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Property 'zookeeper.connection.timeout.ms' is not a valid integer");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
new file mode 100644
index 0000000..21f24e6
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Creates a Kafka consumer compatible with reading from Kafka 0.8.1.x brokers.
+ * The consumer will internally use the old low-level Kafka API, and manually commit offsets
+ * partition offsets to ZooKeeper.
+ * 
+ * <p>The following additional configuration values are available:</p>
+ * <ul>
+ *   <li>socket.timeout.ms</li>
+ *   <li>socket.receive.buffer.bytes</li>
+ *   <li>fetch.message.max.bytes</li>
+ *   <li>auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)</li>
+ *   <li>fetch.wait.max.ms</li>
+ * </ul>
+ * 
+ * @param <T> The type of elements produced by this consumer.
+ */
+public class FlinkKafkaConsumer081<T> extends FlinkKafkaConsumer<T> {
+
+	private static final long serialVersionUID = -5649906773771949146L;
+
+	/**
+	 * Creates a new Kafka 0.8.1.x streaming source consumer.
+	 *
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer081(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
+		super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
new file mode 100644
index 0000000..77e41e5
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.kafka;
+
+import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+
+import java.util.Properties;
+
+/**
+ * Creates a Kafka consumer compatible with reading from Kafka 0.8.2.x brokers.
+ * The consumer will internally use the old low-level Kafka API, and manually commit offsets
+ * partition offsets to ZooKeeper.
+ *
+ * Once Kafka released the new consumer with Kafka 0.8.3 Flink might use the 0.8.3 consumer API
+ * also against Kafka 0.8.2 installations.
+ *
+ * @param <T> The type of elements produced by this consumer.
+ */
+public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumer<T> {
+
+	private static final long serialVersionUID = -8450689820627198228L;
+
+	/**
+	 * Creates a new Kafka 0.8.2.x streaming source consumer.
+	 * 
+	 * @param topic
+	 *           The name of the topic that should be consumed.
+	 * @param valueDeserializer
+	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
+	 * @param props
+	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
+	 */
+	public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
+		super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
+	}
+}


[47/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
deleted file mode 100644
index 42a5951..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
+++ /dev/null
@@ -1,128 +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.connectors.kafka.internals;
-
-import kafka.common.TopicAndPartition;
-import kafka.utils.ZKGroupTopicDirs;
-import kafka.utils.ZkUtils;
-
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.zookeeper.data.Stat;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import scala.Option;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-public class ZookeeperOffsetHandler implements OffsetHandler {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class);
-	
-	private static final long OFFSET_NOT_SET = FlinkKafkaConsumer.OFFSET_NOT_SET;
-	
-	
-	private final ZkClient zkClient;
-	
-	private final String groupId;
-
-	
-	public ZookeeperOffsetHandler(Properties props) {
-		this.groupId = props.getProperty(ConsumerConfig.GROUP_ID_CONFIG);
-		
-		if (this.groupId == null) {
-			throw new IllegalArgumentException("Required property '"
-					+ ConsumerConfig.GROUP_ID_CONFIG + "' has not been set");
-		}
-		
-		String zkConnect = props.getProperty("zookeeper.connect");
-		if (zkConnect == null) {
-			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set");
-		}
-		
-		zkClient = new ZkClient(zkConnect,
-				Integer.valueOf(props.getProperty("zookeeper.session.timeout.ms", "6000")),
-				Integer.valueOf(props.getProperty("zookeeper.connection.timeout.ms", "6000")),
-				new ZooKeeperStringSerializer());
-	}
-
-
-	@Override
-	public void commit(Map<TopicPartition, Long> offsetsToCommit) {
-		for (Map.Entry<TopicPartition, Long> entry : offsetsToCommit.entrySet()) {
-			TopicPartition tp = entry.getKey();
-			long offset = entry.getValue();
-			
-			if (offset >= 0) {
-				setOffsetInZooKeeper(zkClient, groupId, tp.topic(), tp.partition(), offset);
-			}
-		}
-	}
-
-	@Override
-	public void seekFetcherToInitialOffsets(List<TopicPartition> partitions, Fetcher fetcher) {
-		for (TopicPartition tp : partitions) {
-			long offset = getOffsetFromZooKeeper(zkClient, groupId, tp.topic(), tp.partition());
-
-			if (offset != OFFSET_NOT_SET) {
-				LOG.info("Offset for partition {} was set to {} in ZooKeeper. Seeking fetcher to that position.",
-						tp.partition(), offset);
-
-				// the offset in Zookeeper was the last read offset, seek is accepting the next-to-read-offset.
-				fetcher.seek(tp, offset + 1);
-			}
-		}
-	}
-
-	@Override
-	public void close() throws IOException {
-		zkClient.close();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Communication with Zookeeper
-	// ------------------------------------------------------------------------
-	
-	public static void setOffsetInZooKeeper(ZkClient zkClient, String groupId, String topic, int partition, long offset) {
-		TopicAndPartition tap = new TopicAndPartition(topic, partition);
-		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
-		ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir() + "/" + tap.partition(), Long.toString(offset));
-	}
-
-	public static long getOffsetFromZooKeeper(ZkClient zkClient, String groupId, String topic, int partition) {
-		TopicAndPartition tap = new TopicAndPartition(topic, partition);
-		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, tap.topic());
-
-		scala.Tuple2<Option<String>, Stat> data = ZkUtils.readDataMaybeNull(zkClient,
-				topicDirs.consumerOffsetDir() + "/" + tap.partition());
-
-		if (data._1().isEmpty()) {
-			return OFFSET_NOT_SET;
-		} else {
-			return Long.valueOf(data._1().get());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
deleted file mode 100644
index 346a7d5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/FixedPartitioner.java
+++ /dev/null
@@ -1,80 +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.connectors.kafka.partitioner;
-
-import java.io.Serializable;
-
-/**
- * A partitioner ensuring that each internal Flink partition ends up in one Kafka partition.
- *
- * Note, one Kafka partition can contain multiple Flink partitions.
- *
- * Cases:
- * 	# More Flink partitions than kafka partitions
- * <pre>
- * 		Flink Sinks:		Kafka Partitions
- * 			1	---------------->	1
- * 			2   --------------/
- * 			3   -------------/
- * 			4	------------/
- * </pre>
- * 	--> Some (or all) kafka partitions contain the output of more than one flink partition
- *
- *# Fewer Flink partitions than Kafka
- * <pre>
- * 		Flink Sinks:		Kafka Partitions
- * 			1	---------------->	1
- * 			2	---------------->	2
- * 									3
- * 									4
- * 									5
- * </pre>
- *
- *  --> Not all Kafka partitions contain data
- *  To avoid such an unbalanced partitioning, use a round-robin kafka partitioner. (note that this will
- *  cause a lot of network connections between all the Flink instances and all the Kafka brokers
- *
- *
- */
-public class FixedPartitioner extends KafkaPartitioner implements Serializable {
-	private static final long serialVersionUID = 1627268846962918126L;
-
-	int targetPartition = -1;
-
-	@Override
-	public void open(int parallelInstanceId, int parallelInstances, int[] partitions) {
-		int p = 0;
-		for(int i = 0; i < parallelInstances; i++) {
-			if(i == parallelInstanceId) {
-				targetPartition = partitions[p];
-				return;
-			}
-			if(++p == partitions.length) {
-				p = 0;
-			}
-		}
-	}
-
-	@Override
-	public int partition(Object element, int numPartitions) {
-		if(targetPartition == -1) {
-			throw new RuntimeException("The partitioner has not been initialized properly");
-		}
-		return targetPartition;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
deleted file mode 100644
index 55519f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
+++ /dev/null
@@ -1,42 +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.connectors.kafka.partitioner;
-
-
-import kafka.producer.Partitioner;
-
-import java.io.Serializable;
-
-/**
- * Extended Kafka Partitioner.
- * It contains a open() method which is called on each parallel instance.
- * Partitioners have to be serializable!
- */
-public abstract class KafkaPartitioner implements Partitioner, Serializable {
-
-	private static final long serialVersionUID = -1974260817778593473L;
-
-	/**
-	 * Initializer for the Partitioner.
-	 * @param parallelInstanceId 0-indexed id of the parallel instance in Flink
-	 * @param parallelInstances the total number of parallel instances
-	 * @param partitions an array describing the partition IDs of the available Kafka partitions.
-	 */
-	public void open(int parallelInstanceId, int parallelInstances, int[] partitions) {
-		// overwrite this method if needed.
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
deleted file mode 100644
index 3d392aa..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
+++ /dev/null
@@ -1,257 +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.connectors.kafka;
-
-
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.apache.kafka.common.TopicPartition;
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests that the partition assignment is deterministic and stable.
- */
-public class KafkaConsumerPartitionAssignmentTest {
-
-	@Test
-	public void testPartitionsEqualConsumers() {
-		try {
-			int[] partitions = {4, 52, 17, 1};
-			
-			for (int i = 0; i < partitions.length; i++) {
-				List<TopicPartition> parts = FlinkKafkaConsumer.assignPartitions(
-						partitions, "test-topic", partitions.length, i);
-				
-				assertNotNull(parts);
-				assertEquals(1, parts.size());
-				assertTrue(contains(partitions, parts.get(0).partition()));
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testMultiplePartitionsPerConsumers() {
-		try {
-			final int[] partitions = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14};
-
-			final Set<Integer> allPartitions = new HashSet<>();
-			for (int i : partitions) {
-				allPartitions.add(i);
-			}
-			
-			final int numConsumers = 3;
-			final int minPartitionsPerConsumer = partitions.length / numConsumers;
-			final int maxPartitionsPerConsumer = partitions.length / numConsumers + 1;
-			
-			for (int i = 0; i < numConsumers; i++) {
-				List<TopicPartition> parts = FlinkKafkaConsumer.assignPartitions(
-						partitions, "test-topic", numConsumers, i);
-
-				assertNotNull(parts);
-				assertTrue(parts.size() >= minPartitionsPerConsumer);
-				assertTrue(parts.size() <= maxPartitionsPerConsumer);
-
-				for (TopicPartition p : parts) {
-					// check that the element was actually contained
-					assertTrue(allPartitions.remove(p.partition()));
-				}
-			}
-
-			// all partitions must have been assigned
-			assertTrue(allPartitions.isEmpty());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testPartitionsFewerThanConsumers() {
-		try {
-			final int[] partitions = {4, 52, 17, 1};
-
-			final Set<Integer> allPartitions = new HashSet<>();
-			for (int i : partitions) {
-				allPartitions.add(i);
-			}
-
-			final int numConsumers = 2 * partitions.length + 3;
-			
-			for (int i = 0; i < numConsumers; i++) {
-				List<TopicPartition> parts = FlinkKafkaConsumer.assignPartitions(
-						partitions, "test-topic", numConsumers, i);
-
-				assertNotNull(parts);
-				assertTrue(parts.size() <= 1);
-				
-				for (TopicPartition p : parts) {
-					// check that the element was actually contained
-					assertTrue(allPartitions.remove(p.partition()));
-				}
-			}
-
-			// all partitions must have been assigned
-			assertTrue(allPartitions.isEmpty());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testAssignEmptyPartitions() {
-		try {
-			List<TopicPartition> parts1 = FlinkKafkaConsumer.assignPartitions(new int[0], "test-topic", 4, 2);
-			assertNotNull(parts1);
-			assertTrue(parts1.isEmpty());
-
-			List<TopicPartition> parts2 = FlinkKafkaConsumer.assignPartitions(new int[0], "test-topic", 1, 0);
-			assertNotNull(parts2);
-			assertTrue(parts2.isEmpty());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testGrowingPartitionsRemainsStable() {
-		try {
-			final int[] newPartitions = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14};
-			final int[] initialPartitions = Arrays.copyOfRange(newPartitions, 0, 7);
-
-			final Set<Integer> allNewPartitions = new HashSet<>();
-			final Set<Integer> allInitialPartitions = new HashSet<>();
-			for (int i : newPartitions) {
-				allNewPartitions.add(i);
-			}
-			for (int i : initialPartitions) {
-				allInitialPartitions.add(i);
-			}
-
-			final int numConsumers = 3;
-			final int minInitialPartitionsPerConsumer = initialPartitions.length / numConsumers;
-			final int maxInitialPartitionsPerConsumer = initialPartitions.length / numConsumers + 1;
-			final int minNewPartitionsPerConsumer = newPartitions.length / numConsumers;
-			final int maxNewPartitionsPerConsumer = newPartitions.length / numConsumers + 1;
-			
-			List<TopicPartition> parts1 = FlinkKafkaConsumer.assignPartitions(
-					initialPartitions, "test-topic", numConsumers, 0);
-			List<TopicPartition> parts2 = FlinkKafkaConsumer.assignPartitions(
-					initialPartitions, "test-topic", numConsumers, 1);
-			List<TopicPartition> parts3 = FlinkKafkaConsumer.assignPartitions(
-					initialPartitions, "test-topic", numConsumers, 2);
-
-			assertNotNull(parts1);
-			assertNotNull(parts2);
-			assertNotNull(parts3);
-			
-			assertTrue(parts1.size() >= minInitialPartitionsPerConsumer);
-			assertTrue(parts1.size() <= maxInitialPartitionsPerConsumer);
-			assertTrue(parts2.size() >= minInitialPartitionsPerConsumer);
-			assertTrue(parts2.size() <= maxInitialPartitionsPerConsumer);
-			assertTrue(parts3.size() >= minInitialPartitionsPerConsumer);
-			assertTrue(parts3.size() <= maxInitialPartitionsPerConsumer);
-
-			for (TopicPartition p : parts1) {
-				// check that the element was actually contained
-				assertTrue(allInitialPartitions.remove(p.partition()));
-			}
-			for (TopicPartition p : parts2) {
-				// check that the element was actually contained
-				assertTrue(allInitialPartitions.remove(p.partition()));
-			}
-			for (TopicPartition p : parts3) {
-				// check that the element was actually contained
-				assertTrue(allInitialPartitions.remove(p.partition()));
-			}
-			
-			// all partitions must have been assigned
-			assertTrue(allInitialPartitions.isEmpty());
-			
-			// grow the set of partitions and distribute anew
-			
-			List<TopicPartition> parts1new = FlinkKafkaConsumer.assignPartitions(
-					newPartitions, "test-topic", numConsumers, 0);
-			List<TopicPartition> parts2new = FlinkKafkaConsumer.assignPartitions(
-					newPartitions, "test-topic", numConsumers, 1);
-			List<TopicPartition> parts3new = FlinkKafkaConsumer.assignPartitions(
-					newPartitions, "test-topic", numConsumers, 2);
-
-			// new partitions must include all old partitions
-			
-			assertTrue(parts1new.size() > parts1.size());
-			assertTrue(parts2new.size() > parts2.size());
-			assertTrue(parts3new.size() > parts3.size());
-			
-			assertTrue(parts1new.containsAll(parts1));
-			assertTrue(parts2new.containsAll(parts2));
-			assertTrue(parts3new.containsAll(parts3));
-
-			assertTrue(parts1new.size() >= minNewPartitionsPerConsumer);
-			assertTrue(parts1new.size() <= maxNewPartitionsPerConsumer);
-			assertTrue(parts2new.size() >= minNewPartitionsPerConsumer);
-			assertTrue(parts2new.size() <= maxNewPartitionsPerConsumer);
-			assertTrue(parts3new.size() >= minNewPartitionsPerConsumer);
-			assertTrue(parts3new.size() <= maxNewPartitionsPerConsumer);
-
-			for (TopicPartition p : parts1new) {
-				// check that the element was actually contained
-				assertTrue(allNewPartitions.remove(p.partition()));
-			}
-			for (TopicPartition p : parts2new) {
-				// check that the element was actually contained
-				assertTrue(allNewPartitions.remove(p.partition()));
-			}
-			for (TopicPartition p : parts3new) {
-				// check that the element was actually contained
-				assertTrue(allNewPartitions.remove(p.partition()));
-			}
-
-			// all partitions must have been assigned
-			assertTrue(allNewPartitions.isEmpty());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	private static boolean contains(int[] array, int value) {
-		for (int i : array) {
-			if (i == value) {
-				return true;
-			}
-		}
-		return false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
deleted file mode 100644
index e35fcfb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
+++ /dev/null
@@ -1,144 +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.connectors.kafka;
-
-import org.apache.commons.collections.map.LinkedMap;
-
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import java.lang.reflect.Field;
-import java.util.Arrays;
-import java.util.Properties;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-public class KafkaConsumerTest {
-
-	@Test
-	public void testValidateZooKeeperConfig() {
-		try {
-			// empty
-			Properties emptyProperties = new Properties();
-			try {
-				FlinkKafkaConsumer.validateZooKeeperConfig(emptyProperties);
-				fail("should fail with an exception");
-			}
-			catch (IllegalArgumentException e) {
-				// expected
-			}
-
-			// no connect string (only group string)
-			Properties noConnect = new Properties();
-			noConnect.put(ConsumerConfig.GROUP_ID_CONFIG, "flink-test-group");
-			try {
-				FlinkKafkaConsumer.validateZooKeeperConfig(noConnect);
-				fail("should fail with an exception");
-			}
-			catch (IllegalArgumentException e) {
-				// expected
-			}
-
-			// no group string (only connect string)
-			Properties noGroup = new Properties();
-			noGroup.put("zookeeper.connect", "localhost:47574");
-			try {
-				FlinkKafkaConsumer.validateZooKeeperConfig(noGroup);
-				fail("should fail with an exception");
-			}
-			catch (IllegalArgumentException e) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSnapshot() {
-		try {
-			Field offsetsField = FlinkKafkaConsumer.class.getDeclaredField("lastOffsets");
-			Field runningField = FlinkKafkaConsumer.class.getDeclaredField("running");
-			Field mapField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints");
-			
-			offsetsField.setAccessible(true);
-			runningField.setAccessible(true);
-			mapField.setAccessible(true);
-
-			FlinkKafkaConsumer<?> consumer = mock(FlinkKafkaConsumer.class);
-			when(consumer.snapshotState(anyLong(), anyLong())).thenCallRealMethod();
-			
-			long[] testOffsets = new long[] { 43, 6146, 133, 16, 162, 616 };
-			LinkedMap map = new LinkedMap();
-			
-			offsetsField.set(consumer, testOffsets);
-			runningField.set(consumer, true);
-			mapField.set(consumer, map);
-			
-			assertTrue(map.isEmpty());
-
-			// make multiple checkpoints
-			for (long checkpointId = 10L; checkpointId <= 2000L; checkpointId += 9L) {
-				long[] checkpoint = consumer.snapshotState(checkpointId, 47 * checkpointId);
-				assertArrayEquals(testOffsets, checkpoint);
-				
-				// change the offsets, make sure the snapshot did not change
-				long[] checkpointCopy = Arrays.copyOf(checkpoint, checkpoint.length);
-				
-				for (int i = 0; i < testOffsets.length; i++) {
-					testOffsets[i] += 1L;
-				}
-				
-				assertArrayEquals(checkpointCopy, checkpoint);
-				
-				assertTrue(map.size() > 0);
-				assertTrue(map.size() <= FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS);
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	@Ignore("Kafka consumer internally makes an infinite loop")
-	public void testCreateSourceWithoutCluster() {
-		try {
-			Properties props = new Properties();
-			props.setProperty("zookeeper.connect", "localhost:56794");
-			props.setProperty("bootstrap.servers", "localhost:11111, localhost:22222");
-			props.setProperty("group.id", "non-existent-group");
-
-			new FlinkKafkaConsumer<>("no op topic", new JavaDefaultStringSchema(), props,
-					FlinkKafkaConsumer.OffsetStore.FLINK_ZOOKEEPER,
-					FlinkKafkaConsumer.FetcherType.LEGACY_LOW_LEVEL);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
deleted file mode 100644
index e9a5728..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ /dev/null
@@ -1,1124 +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.connectors.kafka;
-
-import kafka.admin.AdminUtils;
-import kafka.api.PartitionMetadata;
-import kafka.consumer.Consumer;
-import kafka.consumer.ConsumerConfig;
-import kafka.consumer.ConsumerIterator;
-import kafka.consumer.KafkaStream;
-import kafka.javaapi.consumer.ConsumerConnector;
-import kafka.message.MessageAndMetadata;
-import kafka.server.KafkaServer;
-
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.commons.collections.map.LinkedMap;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-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.source.RichSourceFunction;
-import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
-import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators;
-import org.apache.flink.streaming.connectors.kafka.testutils.DiscardingSink;
-import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils;
-import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
-import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper;
-import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2Partitioner;
-import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-import org.apache.flink.testutils.junit.RetryOnException;
-import org.apache.flink.testutils.junit.RetryRule;
-import org.apache.flink.util.Collector;
-
-import org.apache.flink.util.NetUtils;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.junit.Assert;
-
-import org.junit.Rule;
-import scala.collection.Seq;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-
-@SuppressWarnings("serial")
-public abstract class KafkaConsumerTestBase extends KafkaTestBase {
-	
-	@Rule
-	public RetryRule retryRule = new RetryRule();
-	
-	// ------------------------------------------------------------------------
-	//  Required methods by the abstract test base
-	// ------------------------------------------------------------------------
-
-	protected abstract <T> FlinkKafkaConsumer<T> getConsumer(
-			String topic, DeserializationSchema<T> deserializationSchema, Properties props);
-
-	// ------------------------------------------------------------------------
-	//  Suite of Tests
-	//
-	//  The tests here are all not activated (by an @Test tag), but need
-	//  to be invoked from the extending classes. That way, the classes can
-	//  select which tests to run.
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Test that validates that checkpointing and checkpoint notification works properly
-	 */
-	public void runCheckpointingTest() throws Exception {
-		createTestTopic("testCheckpointing", 1, 1);
-
-		FlinkKafkaConsumer<String> source = getConsumer("testCheckpointing", new JavaDefaultStringSchema(), standardProps);
-		Field pendingCheckpointsField = FlinkKafkaConsumer.class.getDeclaredField("pendingCheckpoints");
-		pendingCheckpointsField.setAccessible(true);
-		LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source);
-
-		Assert.assertEquals(0, pendingCheckpoints.size());
-		source.setRuntimeContext(new MockRuntimeContext(1, 0));
-
-		final long[] initialOffsets = new long[] { 1337 };
-
-		// first restore
-		source.restoreState(initialOffsets);
-
-		// then open
-		source.open(new Configuration());
-		long[] state1 = source.snapshotState(1, 15);
-
-		assertArrayEquals(initialOffsets, state1);
-
-		long[] state2 = source.snapshotState(2, 30);
-		Assert.assertArrayEquals(initialOffsets, state2);
-		Assert.assertEquals(2, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(1);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(2);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(666); // invalid checkpoint
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		// create 500 snapshots
-		for (int i = 100; i < 600; i++) {
-			source.snapshotState(i, 15 * i);
-		}
-		Assert.assertEquals(FlinkKafkaConsumer.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size());
-
-		// commit only the second last
-		source.notifyCheckpointComplete(598);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		// access invalid checkpoint
-		source.notifyCheckpointComplete(590);
-
-		// and the last
-		source.notifyCheckpointComplete(599);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		source.close();
-
-		deleteTestTopic("testCheckpointing");
-	}
-
-	/**
-	 * Tests that offsets are properly committed to ZooKeeper and initial offsets are read from ZooKeeper.
-	 *
-	 * This test is only applicable if Teh Flink Kafka Consumer uses the ZooKeeperOffsetHandler.
-	 */
-	public void runOffsetInZookeeperValidationTest() throws Exception {
-		LOG.info("Starting testFlinkKafkaConsumerWithOffsetUpdates()");
-
-		final String topicName = "testOffsetHacking";
-		final int parallelism = 3;
-
-		createTestTopic(topicName, parallelism, 1);
-
-		StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env1.getConfig().disableSysoutLogging();
-		env1.enableCheckpointing(50);
-		env1.setNumberOfExecutionRetries(0);
-		env1.setParallelism(parallelism);
-
-		StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env2.getConfig().disableSysoutLogging();
-		env2.enableCheckpointing(50);
-		env2.setNumberOfExecutionRetries(0);
-		env2.setParallelism(parallelism);
-
-		StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env3.getConfig().disableSysoutLogging();
-		env3.enableCheckpointing(50);
-		env3.setNumberOfExecutionRetries(0);
-		env3.setParallelism(parallelism);
-
-		// write a sequence from 0 to 99 to each of the 3 partitions.
-		writeSequence(env1, topicName, 100, parallelism);
-
-		readSequence(env2, standardProps, parallelism, topicName, 100, 0);
-
-		ZkClient zkClient = createZookeeperClient();
-
-		long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 0);
-		long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 1);
-		long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(zkClient, standardCC.groupId(), topicName, 2);
-
-		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
-
-		assertTrue(o1 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
-		assertTrue(o2 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
-		assertTrue(o3 == FlinkKafkaConsumer.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
-
-		LOG.info("Manipulating offsets");
-
-		// set the offset to 50 for the three partitions
-		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 0, 49);
-		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 1, 49);
-		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topicName, 2, 49);
-
-		zkClient.close();
-
-		// create new env
-		readSequence(env3, standardProps, parallelism, topicName, 50, 50);
-
-		deleteTestTopic(topicName);
-
-		LOG.info("Finished testFlinkKafkaConsumerWithOffsetUpdates()");
-	}
-
-	/**
-	 * Ensure Kafka is working on both producer and consumer side.
-	 * This executes a job that contains two Flink pipelines.
-	 *
-	 * <pre>
-	 * (generator source) --> (kafka sink)-[KAFKA-TOPIC]-(kafka source) --> (validating sink)
-	 * </pre>
-	 * 
-	 * We need to externally retry this test. We cannot let Flink's retry mechanism do it, because the Kafka producer
-	 * does not guarantee exactly-once output. Hence a recovery would introduce duplicates that
-	 * cause the test to fail.
-	 */
-	@RetryOnException(times=2, exception=kafka.common.NotLeaderForPartitionException.class)
-	public void runSimpleConcurrentProducerConsumerTopology() throws Exception {
-		LOG.info("Starting runSimpleConcurrentProducerConsumerTopology()");
-
-		final String topic = "concurrentProducerConsumerTopic_" + UUID.randomUUID().toString();
-		final int parallelism = 3;
-		final int elementsPerPartition = 100;
-		final int totalElements = parallelism * elementsPerPartition;
-
-		createTestTopic(topic, parallelism, 2);
-
-		final StreamExecutionEnvironment env =
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(parallelism);
-		env.getConfig().disableSysoutLogging();
-
-		TypeInformation<Tuple2<Long, String>> longStringType = TypeInfoParser.parse("Tuple2<Long, String>");
-
-		TypeInformationSerializationSchema<Tuple2<Long, String>> sourceSchema =
-				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
-
-		TypeInformationSerializationSchema<Tuple2<Long, String>> sinkSchema =
-				new TypeInformationSerializationSchema<>(longStringType, env.getConfig());
-
-		// ----------- add producer dataflow ----------
-
-		DataStream<Tuple2<Long, String>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Long,String>>() {
-
-			private boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, String>> ctx) {
-				int cnt = getRuntimeContext().getIndexOfThisSubtask() * elementsPerPartition;
-				int limit = cnt + elementsPerPartition;
-
-
-				while (running && cnt < limit) {
-					ctx.collect(new Tuple2<Long, String>(1000L + cnt, "kafka-" + cnt));
-					cnt++;
-				}
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		});
-		stream.addSink(new FlinkKafkaProducer<>(brokerConnectionStrings, topic, sinkSchema));
-
-		// ----------- add consumer dataflow ----------
-
-		FlinkKafkaConsumer<Tuple2<Long, String>> source = getConsumer(topic, sourceSchema, standardProps);
-
-		DataStreamSource<Tuple2<Long, String>> consuming = env.addSource(source).setParallelism(parallelism);
-
-		consuming.addSink(new RichSinkFunction<Tuple2<Long, String>>() {
-
-			private int elCnt = 0;
-			private BitSet validator = new BitSet(totalElements);
-
-			@Override
-			public void invoke(Tuple2<Long, String> value) throws Exception {
-				String[] sp = value.f1.split("-");
-				int v = Integer.parseInt(sp[1]);
-
-				assertEquals(value.f0 - 1000, (long) v);
-
-				assertFalse("Received tuple twice", validator.get(v));
-				validator.set(v);
-				elCnt++;
-
-				if (elCnt == totalElements) {
-					// check if everything in the bitset is set to true
-					int nc;
-					if ((nc = validator.nextClearBit(0)) != totalElements) {
-						fail("The bitset was not set to 1 on all elements. Next clear:"
-								+ nc + " Set: " + validator);
-					}
-					throw new SuccessException();
-				}
-			}
-
-			@Override
-			public void close() throws Exception {
-				super.close();
-			}
-		}).setParallelism(1);
-
-		try {
-			tryExecutePropagateExceptions(env, "runSimpleConcurrentProducerConsumerTopology");
-		}
-		catch (ProgramInvocationException | JobExecutionException e) {
-			// look for NotLeaderForPartitionException
-			Throwable cause = e.getCause();
-
-			// search for nested SuccessExceptions
-			int depth = 0;
-			while (cause != null && depth++ < 20) {
-				if (cause instanceof kafka.common.NotLeaderForPartitionException) {
-					throw (Exception) cause;
-				}
-				cause = cause.getCause();
-			}
-			throw e;
-		}
-
-		LOG.info("Finished runSimpleConcurrentProducerConsumerTopology()");
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and
-	 * Flink sources.
-	 */
-	public void runOneToOneExactlyOnceTest() throws Exception {
-		LOG.info("Starting runOneToOneExactlyOnceTest()");
-
-		final String topic = "oneToOneTopic";
-		final int parallelism = 5;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = parallelism * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-		createTestTopic(topic, parallelism, 1);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				brokerConnectionStrings,
-				topic, parallelism, numElementsPerPartition, true);
-
-		// run the topology that fails and recovers
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.enableCheckpointing(500);
-		env.setParallelism(parallelism);
-		env.setNumberOfExecutionRetries(3);
-		env.getConfig().disableSysoutLogging();
-
-		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-
-		env
-				.addSource(kafkaSource)
-				.map(new PartitionValidatingMapper(parallelism, 1))
-				.map(new FailingIdentityMapper<Integer>(failAfterElements))
-				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		FailingIdentityMapper.failedBefore = false;
-		tryExecute(env, "One-to-one exactly once test");
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests the proper consumption when having fewer Flink sources than Kafka partitions, so
-	 * one Flink source will read multiple Kafka partitions.
-	 */
-	public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception {
-		LOG.info("Starting runOneSourceMultiplePartitionsExactlyOnceTest()");
-
-		final String topic = "oneToManyTopic";
-		final int numPartitions = 5;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = numPartitions * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-		final int parallelism = 2;
-
-		createTestTopic(topic, numPartitions, 1);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				brokerConnectionStrings,
-				topic, numPartitions, numElementsPerPartition, true);
-
-		// run the topology that fails and recovers
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.enableCheckpointing(500);
-		env.setParallelism(parallelism);
-		env.setNumberOfExecutionRetries(3);
-		env.getConfig().disableSysoutLogging();
-
-		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-
-		env
-				.addSource(kafkaSource)
-				.map(new PartitionValidatingMapper(numPartitions, 3))
-				.map(new FailingIdentityMapper<Integer>(failAfterElements))
-				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		FailingIdentityMapper.failedBefore = false;
-		tryExecute(env, "One-source-multi-partitions exactly once test");
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests the proper consumption when having more Flink sources than Kafka partitions, which means
-	 * that some Flink sources will read no partitions.
-	 */
-	public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception {
-		LOG.info("Starting runMultipleSourcesOnePartitionExactlyOnceTest()");
-
-		final String topic = "manyToOneTopic";
-		final int numPartitions = 5;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = numPartitions * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-		final int parallelism = 8;
-
-		createTestTopic(topic, numPartitions, 1);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				brokerConnectionStrings,
-				topic, numPartitions, numElementsPerPartition, true);
-
-		// run the topology that fails and recovers
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.enableCheckpointing(500);
-		env.setParallelism(parallelism);
-		env.setNumberOfExecutionRetries(3);
-		env.getConfig().disableSysoutLogging();
-		env.setBufferTimeout(0);
-
-		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-
-		env
-			.addSource(kafkaSource)
-			.map(new PartitionValidatingMapper(numPartitions, 1))
-			.map(new FailingIdentityMapper<Integer>(failAfterElements))
-			.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		FailingIdentityMapper.failedBefore = false;
-		tryExecute(env, "multi-source-one-partitions exactly once test");
-
-
-		deleteTestTopic(topic);
-	}
-	
-	
-	/**
-	 * Tests that the source can be properly canceled when reading full partitions. 
-	 */
-	public void runCancelingOnFullInputTest() throws Exception {
-		final String topic = "cancelingOnFullTopic";
-
-		final int parallelism = 3;
-		createTestTopic(topic, parallelism, 1);
-
-		// launch a producer thread
-		DataGenerators.InfiniteStringsGenerator generator =
-				new DataGenerators.InfiniteStringsGenerator(brokerConnectionStrings, topic);
-		generator.start();
-
-		// launch a consumer asynchronously
-
-		final AtomicReference<Throwable> jobError = new AtomicReference<>();
-
-		final Runnable jobRunner = new Runnable() {
-			@Override
-			public void run() {
-				try {
-					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-					env.setParallelism(parallelism);
-					env.enableCheckpointing(100);
-					env.getConfig().disableSysoutLogging();
-
-					FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
-
-					env.addSource(source).addSink(new DiscardingSink<String>());
-
-					env.execute();
-				}
-				catch (Throwable t) {
-					jobError.set(t);
-				}
-			}
-		};
-
-		Thread runnerThread = new Thread(jobRunner, "program runner thread");
-		runnerThread.start();
-
-		// wait a bit before canceling
-		Thread.sleep(2000);
-
-		// cancel
-		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
-
-		// wait for the program to be done and validate that we failed with the right exception
-		runnerThread.join();
-
-		Throwable failueCause = jobError.get();
-		assertNotNull("program did not fail properly due to canceling", failueCause);
-		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
-
-		if (generator.isAlive()) {
-			generator.shutdown();
-			generator.join();
-		}
-		else {
-			Throwable t = generator.getError();
-			if (t != null) {
-				t.printStackTrace();
-				fail("Generator failed: " + t.getMessage());
-			} else {
-				fail("Generator failed with no exception");
-			}
-		}
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests that the source can be properly canceled when reading empty partitions. 
-	 */
-	public void runCancelingOnEmptyInputTest() throws Exception {
-		final String topic = "cancelingOnEmptyInputTopic";
-
-		final int parallelism = 3;
-		createTestTopic(topic, parallelism, 1);
-
-		final AtomicReference<Throwable> error = new AtomicReference<>();
-
-		final Runnable jobRunner = new Runnable() {
-			@Override
-			public void run() {
-				try {
-					final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-					env.setParallelism(parallelism);
-					env.enableCheckpointing(100);
-					env.getConfig().disableSysoutLogging();
-
-					FlinkKafkaConsumer<String> source = getConsumer(topic, new JavaDefaultStringSchema(), standardProps);
-
-					env.addSource(source).addSink(new DiscardingSink<String>());
-
-					env.execute();
-				}
-				catch (Throwable t) {
-					error.set(t);
-				}
-			}
-		};
-
-		Thread runnerThread = new Thread(jobRunner, "program runner thread");
-		runnerThread.start();
-
-		// wait a bit before canceling
-		Thread.sleep(2000);
-
-		// cancel
-		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
-
-		// wait for the program to be done and validate that we failed with the right exception
-		runnerThread.join();
-
-		Throwable failueCause = error.get();
-		assertNotNull("program did not fail properly due to canceling", failueCause);
-		assertTrue(failueCause.getMessage().contains("Job was cancelled"));
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Tests that the source can be properly canceled when reading full partitions. 
-	 */
-	public void runFailOnDeployTest() throws Exception {
-		final String topic = "failOnDeployTopic";
-
-		createTestTopic(topic, 2, 1);
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(12); // needs to be more that the mini cluster has slots
-		env.getConfig().disableSysoutLogging();
-
-		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-
-		env
-				.addSource(kafkaSource)
-				.addSink(new DiscardingSink<Integer>());
-
-		try {
-			env.execute();
-			fail("this test should fail with an exception");
-		}
-		catch (ProgramInvocationException e) {
-
-			// validate that we failed due to a NoResourceAvailableException
-			Throwable cause = e.getCause();
-			int depth = 0;
-			boolean foundResourceException = false;
-
-			while (cause != null && depth++ < 20) {
-				if (cause instanceof NoResourceAvailableException) {
-					foundResourceException = true;
-					break;
-				}
-				cause = cause.getCause();
-			}
-
-			assertTrue("Wrong exception", foundResourceException);
-		}
-
-		deleteTestTopic(topic);
-	}
-
-	public void runInvalidOffsetTest() throws Exception {
-		final String topic = "invalidOffsetTopic";
-		final int parallelism = 1;
-
-		// create topic
-		createTestTopic(topic, parallelism, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-
-		// write 20 messages into topic:
-		writeSequence(env, topic, 20, parallelism);
-
-		// set invalid offset:
-		ZkClient zkClient = createZookeeperClient();
-		ZookeeperOffsetHandler.setOffsetInZooKeeper(zkClient, standardCC.groupId(), topic, 0, 1234);
-
-		// read from topic
-		final int valuesCount = 20;
-		final int startFrom = 0;
-		readSequence(env, standardCC.props().props(), parallelism, topic, valuesCount, startFrom);
-
-		deleteTestTopic(topic);
-	}
-
-	/**
-	 * Test Flink's Kafka integration also with very big records (30MB)
-	 * see http://stackoverflow.com/questions/21020347/kafka-sending-a-15mb-message
-	 */
-	public void runBigRecordTestTopology() throws Exception {
-		LOG.info("Starting runBigRecordTestTopology()");
-
-		final String topic = "bigRecordTestTopic";
-		final int parallelism = 1; // otherwise, the kafka mini clusters may run out of heap space
-
-		createTestTopic(topic, parallelism, 1);
-
-		final TypeInformation<Tuple2<Long, byte[]>> longBytesInfo = TypeInfoParser.parse("Tuple2<Long, byte[]>");
-
-		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> serSchema =
-				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
-
-		final TypeInformationSerializationSchema<Tuple2<Long, byte[]>> deserSchema =
-				new TypeInformationSerializationSchema<>(longBytesInfo, new ExecutionConfig());
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setNumberOfExecutionRetries(0);
-		env.getConfig().disableSysoutLogging();
-		env.enableCheckpointing(100);
-		env.setParallelism(parallelism);
-
-		// add consuming topology:
-		Properties consumerProps = new Properties();
-		consumerProps.putAll(standardProps);
-		consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 40));
-		consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 40)); // for the new fetcher
-		consumerProps.setProperty("queued.max.message.chunks", "1");
-
-		FlinkKafkaConsumer<Tuple2<Long, byte[]>> source = getConsumer(topic, serSchema, consumerProps);
-		DataStreamSource<Tuple2<Long, byte[]>> consuming = env.addSource(source);
-
-		consuming.addSink(new SinkFunction<Tuple2<Long, byte[]>>() {
-
-			private int elCnt = 0;
-
-			@Override
-			public void invoke(Tuple2<Long, byte[]> value) throws Exception {
-				elCnt++;
-				if (value.f0 == -1) {
-					// we should have seen 11 elements now.
-					if(elCnt == 11) {
-						throw new SuccessException();
-					} else {
-						throw new RuntimeException("There have been "+elCnt+" elements");
-					}
-				}
-				if(elCnt > 10) {
-					throw new RuntimeException("More than 10 elements seen: "+elCnt);
-				}
-			}
-		});
-
-		// add producing topology
-		Properties producerProps = new Properties();
-		producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 30));
-		producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerConnectionStrings);
-
-		DataStream<Tuple2<Long, byte[]>> stream = env.addSource(new RichSourceFunction<Tuple2<Long, byte[]>>() {
-
-			private boolean running;
-
-			@Override
-			public void open(Configuration parameters) throws Exception {
-				super.open(parameters);
-				running = true;
-			}
-
-			@Override
-			public void run(SourceContext<Tuple2<Long, byte[]>> ctx) throws Exception {
-				Random rnd = new Random();
-				long cnt = 0;
-				int fifteenMb = 1024 * 1024 * 15;
-
-				while (running) {
-					byte[] wl = new byte[fifteenMb + rnd.nextInt(fifteenMb)];
-					ctx.collect(new Tuple2<Long, byte[]>(cnt++, wl));
-
-					Thread.sleep(100);
-
-					if (cnt == 10) {
-						// signal end
-						ctx.collect(new Tuple2<Long, byte[]>(-1L, new byte[]{1}));
-						break;
-					}
-				}
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		});
-
-		stream.addSink(new FlinkKafkaProducer<>(topic, deserSchema, producerProps));
-
-		tryExecute(env, "big topology test");
-
-		deleteTestTopic(topic);
-
-		LOG.info("Finished runBigRecordTestTopology()");
-	}
-
-	
-	public void runBrokerFailureTest() throws Exception {
-		LOG.info("starting runBrokerFailureTest()");
-
-		final String topic = "brokerFailureTestTopic";
-
-		final int parallelism = 2;
-		final int numElementsPerPartition = 1000;
-		final int totalElements = parallelism * numElementsPerPartition;
-		final int failAfterElements = numElementsPerPartition / 3;
-
-
-		createTestTopic(topic, parallelism, 2);
-
-		DataGenerators.generateRandomizedIntegerSequence(
-				StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort),
-				brokerConnectionStrings,
-				topic, parallelism, numElementsPerPartition, true);
-
-		// find leader to shut down
-		ZkClient zkClient = createZookeeperClient();
-		PartitionMetadata firstPart = null;
-		do {
-			if (firstPart != null) {
-				LOG.info("Unable to find leader. error code {}", firstPart.errorCode());
-				// not the first try. Sleep a bit
-				Thread.sleep(150);
-			}
-
-			Seq<PartitionMetadata> partitionMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkClient).partitionsMetadata();
-			firstPart = partitionMetadata.head();
-		}
-		while (firstPart.errorCode() != 0);
-		zkClient.close();
-
-		final kafka.cluster.Broker leaderToShutDown = firstPart.leader().get();
-		final String leaderToShutDownConnection = 
-				NetUtils.hostAndPortToUrlString(leaderToShutDown.host(), leaderToShutDown.port());
-		
-		
-		final int leaderIdToShutDown = firstPart.leader().get().id();
-		LOG.info("Leader to shutdown {}", leaderToShutDown);
-
-
-		// run the topology that fails and recovers
-
-		DeserializationSchema<Integer> schema =
-				new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig());
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(parallelism);
-		env.enableCheckpointing(500);
-		env.setNumberOfExecutionRetries(3);
-		env.getConfig().disableSysoutLogging();
-
-
-		FlinkKafkaConsumer<Integer> kafkaSource = getConsumer(topic, schema, standardProps);
-
-		env
-				.addSource(kafkaSource)
-				.map(new PartitionValidatingMapper(parallelism, 1))
-				.map(new BrokerKillingMapper<Integer>(leaderToShutDownConnection, failAfterElements))
-				.addSink(new ValidatingExactlyOnceSink(totalElements)).setParallelism(1);
-
-		BrokerKillingMapper.killedLeaderBefore = false;
-		tryExecute(env, "One-to-one exactly once test");
-
-		// start a new broker:
-		brokers.set(leaderIdToShutDown, getKafkaServer(leaderIdToShutDown, tmpKafkaDirs.get(leaderIdToShutDown), kafkaHost, zookeeperConnectionString));
-
-		LOG.info("finished runBrokerFailureTest()");
-	}
-
-	// ------------------------------------------------------------------------
-	//  Reading writing test data sets
-	// ------------------------------------------------------------------------
-
-	private void readSequence(StreamExecutionEnvironment env, Properties cc,
-								final int sourceParallelism,
-								final String topicName,
-								final int valuesCount, final int startFrom) throws Exception {
-
-		final int finalCount = valuesCount * sourceParallelism;
-
-		final TypeInformation<Tuple2<Integer, Integer>> intIntTupleType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-
-		final TypeInformationSerializationSchema<Tuple2<Integer, Integer>> deser =
-				new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig());
-
-		// create the consumer
-		FlinkKafkaConsumer<Tuple2<Integer, Integer>> consumer = getConsumer(topicName, deser, cc);
-
-		DataStream<Tuple2<Integer, Integer>> source = env
-				.addSource(consumer).setParallelism(sourceParallelism)
-				.map(new ThrottledMapper<Tuple2<Integer, Integer>>(20)).setParallelism(sourceParallelism);
-
-		// verify data
-		source.flatMap(new RichFlatMapFunction<Tuple2<Integer, Integer>, Integer>() {
-
-			private int[] values = new int[valuesCount];
-			private int count = 0;
-
-			@Override
-			public void flatMap(Tuple2<Integer, Integer> value, Collector<Integer> out) throws Exception {
-				values[value.f1 - startFrom]++;
-				count++;
-
-				// verify if we've seen everything
-				if (count == finalCount) {
-					for (int i = 0; i < values.length; i++) {
-						int v = values[i];
-						if (v != sourceParallelism) {
-							printTopic(topicName, valuesCount, deser);
-							throw new RuntimeException("Expected v to be 3, but was " + v + " on element " + i + " array=" + Arrays.toString(values));
-						}
-					}
-					// test has passed
-					throw new SuccessException();
-				}
-			}
-
-		}).setParallelism(1);
-
-		tryExecute(env, "Read data from Kafka");
-
-		LOG.info("Successfully read sequence for verification");
-	}
-
-	private static void writeSequence(StreamExecutionEnvironment env, String topicName, final int numElements, int parallelism) throws Exception {
-
-		TypeInformation<Tuple2<Integer, Integer>> resultType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
-
-		DataStream<Tuple2<Integer, Integer>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
-
-			private boolean running = true;
-
-			@Override
-			public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-				int cnt = 0;
-				int partition = getRuntimeContext().getIndexOfThisSubtask();
-
-				while (running && cnt < numElements) {
-					ctx.collect(new Tuple2<Integer, Integer>(partition, cnt));
-					cnt++;
-				}
-			}
-
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		}).setParallelism(parallelism);
-		
-		stream.addSink(new FlinkKafkaProducer<>(topicName,
-				new TypeInformationSerializationSchema<>(resultType, env.getConfig()),
-				FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings),
-				new Tuple2Partitioner(parallelism)
-		)).setParallelism(parallelism);
-
-		env.execute("Write sequence");
-
-		LOG.info("Finished writing sequence");
-	}
-
-	// ------------------------------------------------------------------------
-	//  Debugging utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Read topic to list, only using Kafka code.
-	 */
-	private static List<MessageAndMetadata<byte[], byte[]>> readTopicToList(String topicName, ConsumerConfig config, final int stopAfter) {
-		ConsumerConnector consumerConnector = Consumer.createJavaConsumerConnector(config);
-		// we request only one stream per consumer instance. Kafka will make sure that each consumer group
-		// will see each message only once.
-		Map<String,Integer> topicCountMap = Collections.singletonMap(topicName, 1);
-		Map<String, List<KafkaStream<byte[], byte[]>>> streams = consumerConnector.createMessageStreams(topicCountMap);
-		if(streams.size() != 1) {
-			throw new RuntimeException("Expected only one message stream but got "+streams.size());
-		}
-		List<KafkaStream<byte[], byte[]>> kafkaStreams = streams.get(topicName);
-		if(kafkaStreams == null) {
-			throw new RuntimeException("Requested stream not available. Available streams: "+streams.toString());
-		}
-		if(kafkaStreams.size() != 1) {
-			throw new RuntimeException("Requested 1 stream from Kafka, bot got "+kafkaStreams.size()+" streams");
-		}
-		LOG.info("Opening Consumer instance for topic '{}' on group '{}'", topicName, config.groupId());
-		ConsumerIterator<byte[], byte[]> iteratorToRead = kafkaStreams.get(0).iterator();
-
-		List<MessageAndMetadata<byte[], byte[]>> result = new ArrayList<MessageAndMetadata<byte[], byte[]>>();
-		int read = 0;
-		while(iteratorToRead.hasNext()) {
-			read++;
-			result.add(iteratorToRead.next());
-			if(read == stopAfter) {
-				LOG.info("Read "+read+" elements");
-				return result;
-			}
-		}
-		return result;
-	}
-
-	private static void printTopic(String topicName, ConsumerConfig config,
-								DeserializationSchema<?> deserializationSchema,
-								int stopAfter) {
-
-		List<MessageAndMetadata<byte[], byte[]>> contents = readTopicToList(topicName, config, stopAfter);
-		LOG.info("Printing contents of topic {} in consumer grouo {}", topicName, config.groupId());
-
-		for (MessageAndMetadata<byte[], byte[]> message: contents) {
-			Object out = deserializationSchema.deserialize(message.message());
-			LOG.info("Message: partition: {} offset: {} msg: {}", message.partition(), message.offset(), out.toString());
-		}
-	}
-
-	private static void printTopic(String topicName, int elements,DeserializationSchema<?> deserializer) {
-		// write the sequence to log for debugging purposes
-		Properties stdProps = standardCC.props().props();
-		Properties newProps = new Properties(stdProps);
-		newProps.setProperty("group.id", "topic-printer"+ UUID.randomUUID().toString());
-		newProps.setProperty("auto.offset.reset", "smallest");
-		newProps.setProperty("zookeeper.connect", standardCC.zkConnect());
-
-		ConsumerConfig printerConfig = new ConsumerConfig(newProps);
-		printTopic(topicName, printerConfig, deserializer, elements);
-	}
-
-
-	public static class BrokerKillingMapper<T> extends RichMapFunction<T,T>
-			implements Checkpointed<Integer>, CheckpointNotifier {
-
-		private static final long serialVersionUID = 6334389850158707313L;
-
-		public static volatile boolean killedLeaderBefore;
-		public static volatile boolean hasBeenCheckpointedBeforeFailure;
-		
-		private final String leaderToShutDown;
-		private final int failCount;
-		private int numElementsTotal;
-
-		private boolean failer;
-		private boolean hasBeenCheckpointed;
-
-
-		public BrokerKillingMapper(String leaderToShutDown, int failCount) {
-			this.leaderToShutDown = leaderToShutDown;
-			this.failCount = failCount;
-		}
-
-		@Override
-		public void open(Configuration parameters) {
-			failer = getRuntimeContext().getIndexOfThisSubtask() == 0;
-		}
-
-		@Override
-		public T map(T value) throws Exception {
-			numElementsTotal++;
-			
-			if (!killedLeaderBefore) {
-				Thread.sleep(10);
-				
-				if (failer && numElementsTotal >= failCount) {
-					// shut down a Kafka broker
-					KafkaServer toShutDown = null;
-					for (KafkaServer kafkaServer : brokers) {
-						String connectionUrl = 
-								NetUtils.hostAndPortToUrlString(
-										kafkaServer.config().advertisedHostName(),
-										kafkaServer.config().advertisedPort());
-						if (leaderToShutDown.equals(connectionUrl)) {
-							toShutDown = kafkaServer;
-							break;
-						}
-					}
-	
-					if (toShutDown == null) {
-						StringBuilder listOfBrokers = new StringBuilder();
-						for (KafkaServer kafkaServer : brokers) {
-							listOfBrokers.append(
-									NetUtils.hostAndPortToUrlString(
-											kafkaServer.config().advertisedHostName(),
-											kafkaServer.config().advertisedPort()));
-							listOfBrokers.append(" ; ");
-						}
-						
-						throw new Exception("Cannot find broker to shut down: " + leaderToShutDown
-								+ " ; available brokers: " + listOfBrokers.toString());
-					}
-					else {
-						hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed;
-						killedLeaderBefore = true;
-						toShutDown.shutdown();
-					}
-				}
-			}
-			return value;
-		}
-
-		@Override
-		public void notifyCheckpointComplete(long checkpointId) {
-			hasBeenCheckpointed = true;
-		}
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return numElementsTotal;
-		}
-
-		@Override
-		public void restoreState(Integer state) {
-			this.numElementsTotal = state;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
deleted file mode 100644
index b4511ce..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import org.junit.Test;
-
-import java.util.Properties;
-
-
-public class KafkaITCase extends KafkaConsumerTestBase {
-	
-	@Override
-	protected <T> FlinkKafkaConsumer<T> getConsumer(String topic, DeserializationSchema<T> deserializationSchema, Properties props) {
-		return new FlinkKafkaConsumer081<>(topic, deserializationSchema, props);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Suite of Tests
-	// ------------------------------------------------------------------------
-	
-	@Test
-	public void testCheckpointing() throws Exception {
-		runCheckpointingTest();
-	}
-
-	@Test
-	public void testOffsetInZookeeper() throws Exception {
-		runOffsetInZookeeperValidationTest();
-	}
-	
-	@Test
-	public void testConcurrentProducerConsumerTopology() throws Exception {
-		runSimpleConcurrentProducerConsumerTopology();
-	}
-
-	// --- canceling / failures ---
-	
-	@Test
-	public void testCancelingEmptyTopic() throws Exception {
-		runCancelingOnEmptyInputTest();
-	}
-
-	@Test
-	public void testCancelingFullTopic() throws Exception {
-		runCancelingOnFullInputTest();
-	}
-
-	@Test
-	public void testFailOnDeploy() throws Exception {
-		runFailOnDeployTest();
-	}
-
-	@Test
-	public void testInvalidOffset() throws Exception {
-		runInvalidOffsetTest();
-	}
-
-	// --- source to partition mappings and exactly once ---
-	
-	@Test
-	public void testOneToOneSources() throws Exception {
-		runOneToOneExactlyOnceTest();
-	}
-
-	@Test
-	public void testOneSourceMultiplePartitions() throws Exception {
-		runOneSourceMultiplePartitionsExactlyOnceTest();
-	}
-
-	@Test
-	public void testMultipleSourcesOnePartition() throws Exception {
-		runMultipleSourcesOnePartitionExactlyOnceTest();
-	}
-
-	// --- broker failure ---
-
-	@Test
-	public void testBrokerFailure() throws Exception {
-		runBrokerFailureTest();
-	}
-
-	// --- special executions ---
-	
-	@Test
-	public void testBigRecordJob() throws Exception {
-		runBigRecordTestTopology();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
deleted file mode 100644
index 72d2772..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaLocalSystemTime.java
+++ /dev/null
@@ -1,48 +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.connectors.kafka;
-
-import kafka.utils.Time;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class KafkaLocalSystemTime implements Time {
-
-	private static final Logger LOG = LoggerFactory.getLogger(KafkaLocalSystemTime.class);
-
-	@Override
-	public long milliseconds() {
-		return System.currentTimeMillis();
-	}
-
-	@Override
-	public long nanoseconds() {
-		return System.nanoTime();
-	}
-
-	@Override
-	public void sleep(long ms) {
-		try {
-			Thread.sleep(ms);
-		} catch (InterruptedException e) {
-			LOG.warn("Interruption", e);
-		}
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java
deleted file mode 100644
index 5001364..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerITCase.java
+++ /dev/null
@@ -1,188 +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.connectors.kafka;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.streaming.api.datastream.DataStream;
-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.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.connectors.kafka.testutils.SuccessException;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
-
-import org.junit.Test;
-
-import java.io.Serializable;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-@SuppressWarnings("serial")
-public class KafkaProducerITCase extends KafkaTestBase {
-
-
-	/**
-	 * 
-	 * <pre>
-	 *             +------> (sink) --+--> [KAFKA-1] --> (source) -> (map) --+
-	 *            /                  |                                       \
-	 *           /                   |                                        \
-	 * (source) ----------> (sink) --+--> [KAFKA-2] --> (source) -> (map) -----+-> (sink)
-	 *           \                   |                                        /
-	 *            \                  |                                       /
-	 *             +------> (sink) --+--> [KAFKA-3] --> (source) -> (map) --+
-	 * </pre>
-	 * 
-	 * The mapper validates that the values come consistently from the correct Kafka partition.
-	 * 
-	 * The final sink validates that there are no duplicates and that all partitions are present.
-	 */
-	@Test
-	public void testCustomPartitioning() {
-		try {
-			LOG.info("Starting KafkaProducerITCase.testCustomPartitioning()");
-
-			final String topic = "customPartitioningTestTopic";
-			final int parallelism = 3;
-			
-			createTestTopic(topic, parallelism, 1);
-
-			TypeInformation<Tuple2<Long, String>> longStringInfo = TypeInfoParser.parse("Tuple2<Long, String>");
-
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-			env.setNumberOfExecutionRetries(0);
-			env.getConfig().disableSysoutLogging();
-
-			TypeInformationSerializationSchema<Tuple2<Long, String>> serSchema =
-					new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig());
-
-			TypeInformationSerializationSchema<Tuple2<Long, String>> deserSchema =
-					new TypeInformationSerializationSchema<>(longStringInfo, env.getConfig());
-
-			// ------ producing topology ---------
-			
-			// source has DOP 1 to make sure it generates no duplicates
-			DataStream<Tuple2<Long, String>> stream = env.addSource(new SourceFunction<Tuple2<Long, String>>() {
-
-				private boolean running = true;
-
-				@Override
-				public void run(SourceContext<Tuple2<Long, String>> ctx) throws Exception {
-					long cnt = 0;
-					while (running) {
-						ctx.collect(new Tuple2<Long, String>(cnt, "kafka-" + cnt));
-						cnt++;
-					}
-				}
-
-				@Override
-				public void cancel() {
-					running = false;
-				}
-			})
-			.setParallelism(1);
-			
-			// sink partitions into 
-			stream.addSink(new FlinkKafkaProducer<>(topic, serSchema, FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnectionStrings), new CustomPartitioner(parallelism)))
-			.setParallelism(parallelism);
-
-			// ------ consuming topology ---------
-			
-			FlinkKafkaConsumer<Tuple2<Long, String>> source = 
-					new FlinkKafkaConsumer<>(topic, deserSchema, standardProps, 
-							FlinkKafkaConsumer.OffsetStore.FLINK_ZOOKEEPER,
-							FlinkKafkaConsumer.FetcherType.LEGACY_LOW_LEVEL);
-			
-			env.addSource(source).setParallelism(parallelism)
-
-					// mapper that validates partitioning and maps to partition
-					.map(new RichMapFunction<Tuple2<Long, String>, Integer>() {
-						
-						private int ourPartition = -1;
-						@Override
-						public Integer map(Tuple2<Long, String> value) {
-							int partition = value.f0.intValue() % parallelism;
-							if (ourPartition != -1) {
-								assertEquals("inconsistent partitioning", ourPartition, partition);
-							} else {
-								ourPartition = partition;
-							}
-							return partition;
-						}
-					}).setParallelism(parallelism)
-					
-					.addSink(new SinkFunction<Integer>() {
-						
-						private int[] valuesPerPartition = new int[parallelism];
-						
-						@Override
-						public void invoke(Integer value) throws Exception {
-							valuesPerPartition[value]++;
-							
-							boolean missing = false;
-							for (int i : valuesPerPartition) {
-								if (i < 100) {
-									missing = true;
-									break;
-								}
-							}
-							if (!missing) {
-								throw new SuccessException();
-							}
-						}
-					}).setParallelism(1);
-			
-			tryExecute(env, "custom partitioning test");
-
-			deleteTestTopic(topic);
-			
-			LOG.info("Finished KafkaProducerITCase.testCustomPartitioning()");
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	
-	// ------------------------------------------------------------------------
-
-	public static class CustomPartitioner extends KafkaPartitioner implements Serializable {
-
-		private final int expectedPartitions;
-
-		public CustomPartitioner(int expectedPartitions) {
-			this.expectedPartitions = expectedPartitions;
-		}
-
-		@Override
-		public int partition(Object key, int numPartitions) {
-			@SuppressWarnings("unchecked")
-			Tuple2<Long, String> tuple = (Tuple2<Long, String>) key;
-			
-			assertEquals(expectedPartitions, numPartitions);
-			
-			return (int) (tuple.f0 % numPartitions);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
deleted file mode 100644
index c5c3387..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema;
-import org.apache.flink.util.TestLogger;
-
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.PartitionInfo;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.util.Arrays;
-import java.util.Properties;
-import java.util.concurrent.Future;
-
-
-import static org.mockito.Mockito.*;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-import static org.junit.Assert.*;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(FlinkKafkaProducer.class)
-public class KafkaProducerTest extends TestLogger {
-	
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testPropagateExceptions() {
-		try {
-			// mock kafka producer
-			KafkaProducer<?, ?> kafkaProducerMock = mock(KafkaProducer.class);
-			
-			// partition setup
-			when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
-					Arrays.asList(new PartitionInfo("mock_topic", 42, null, null, null)));
-
-			// failure when trying to send an element
-			when(kafkaProducerMock.send(any(ProducerRecord.class), any(Callback.class)))
-				.thenAnswer(new Answer<Future<RecordMetadata>>() {
-					@Override
-					public Future<RecordMetadata> answer(InvocationOnMock invocation) throws Throwable {
-						Callback callback = (Callback) invocation.getArguments()[1];
-						callback.onCompletion(null, new Exception("Test error"));
-						return null;
-					}
-				});
-			
-			// make sure the FlinkKafkaProducer instantiates our mock producer
-			whenNew(KafkaProducer.class).withAnyArguments().thenReturn(kafkaProducerMock);
-			
-			// (1) producer that propagates errors
-			
-			FlinkKafkaProducer<String> producerPropagating = new FlinkKafkaProducer<String>(
-					"mock_topic", new JavaDefaultStringSchema(), new Properties(), null);
-
-			producerPropagating.setRuntimeContext(new MockRuntimeContext(17, 3));
-			producerPropagating.open(new Configuration());
-			
-			try {
-				producerPropagating.invoke("value");
-				producerPropagating.invoke("value");
-				fail("This should fail with an exception");
-			}
-			catch (Exception e) {
-				assertNotNull(e.getCause());
-				assertNotNull(e.getCause().getMessage());
-				assertTrue(e.getCause().getMessage().contains("Test error"));
-			}
-
-			// (2) producer that only logs errors
-			
-			FlinkKafkaProducer<String> producerLogging = new FlinkKafkaProducer<String>(
-					"mock_topic", new JavaDefaultStringSchema(), new Properties(), null);
-			producerLogging.setLogFailuresOnly(true);
-			
-			producerLogging.setRuntimeContext(new MockRuntimeContext(17, 3));
-			producerLogging.open(new Configuration());
-
-			producerLogging.invoke("value");
-			producerLogging.invoke("value");
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}


[12/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java b/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
new file mode 100644
index 0000000..33a2e47
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
@@ -0,0 +1,205 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.elasticsearch;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.elasticsearch.action.get.GetRequest;
+import org.elasticsearch.action.get.GetResponse;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.Requests;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.common.transport.LocalTransportAddress;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.node.Node;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
+
+public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
+
+	private static final int NUM_ELEMENTS = 20;
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Test
+	public void testNodeClient() throws Exception{
+
+		File dataDir = tempFolder.newFolder();
+
+		Node node = nodeBuilder()
+				.settings(ImmutableSettings.settingsBuilder()
+						.put("http.enabled", false)
+						.put("path.data", dataDir.getAbsolutePath()))
+				// set a custom cluster name to verify that user config works correctly
+				.clusterName("my-node-client-cluster")
+				.local(true)
+				.node();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
+
+		Map<String, String> config = Maps.newHashMap();
+		// This instructs the sink to emit after every element, otherwise they would be buffered
+		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+		config.put("cluster.name", "my-node-client-cluster");
+
+		// connect to our local node
+		config.put("node.local", "true");
+
+		source.addSink(new ElasticsearchSink<>(config, new TestIndexRequestBuilder()));
+
+		env.execute("Elasticsearch Node Client Test");
+
+
+		// verify the results
+		Client client = node.client();
+		for (int i = 0; i < NUM_ELEMENTS; i++) {
+			GetResponse response = client.get(new GetRequest("my-index",
+					"my-type",
+					Integer.toString(i))).actionGet();
+			Assert.assertEquals("message #" + i, response.getSource().get("data"));
+		}
+
+		node.close();
+	}
+
+	@Test
+	public void testTransportClient() throws Exception {
+
+		File dataDir = tempFolder.newFolder();
+
+		Node node = nodeBuilder()
+				.settings(ImmutableSettings.settingsBuilder()
+						.put("http.enabled", false)
+						.put("path.data", dataDir.getAbsolutePath()))
+						// set a custom cluster name to verify that user config works correctly
+				.clusterName("my-node-client-cluster")
+				.local(true)
+				.node();
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
+
+		Map<String, String> config = Maps.newHashMap();
+		// This instructs the sink to emit after every element, otherwise they would be buffered
+		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+		config.put("cluster.name", "my-node-client-cluster");
+
+		// connect to our local node
+		config.put("node.local", "true");
+
+		List<TransportAddress> transports = Lists.newArrayList();
+		transports.add(new LocalTransportAddress("1"));
+
+		source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
+
+		env.execute("Elasticsearch TransportClient Test");
+
+
+		// verify the results
+		Client client = node.client();
+		for (int i = 0; i < NUM_ELEMENTS; i++) {
+			GetResponse response = client.get(new GetRequest("my-index",
+					"my-type",
+					Integer.toString(i))).actionGet();
+			Assert.assertEquals("message #" + i, response.getSource().get("data"));
+		}
+
+		node.close();
+	}
+
+	@Test(expected = JobExecutionException.class)
+	public void testTransportClientFails() throws Exception{
+		// this checks whether the TransportClient fails early when there is no cluster to
+		// connect to. We don't hava such as test for the Node Client version since that
+		// one will block and wait for a cluster to come online
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction());
+
+		Map<String, String> config = Maps.newHashMap();
+		// This instructs the sink to emit after every element, otherwise they would be buffered
+		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+		config.put("cluster.name", "my-node-client-cluster");
+
+		// connect to our local node
+		config.put("node.local", "true");
+
+		List<TransportAddress> transports = Lists.newArrayList();
+		transports.add(new LocalTransportAddress("1"));
+
+		source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
+
+		env.execute("Elasticsearch Node Client Test");
+	}
+
+	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		private volatile boolean running = true;
+
+		@Override
+		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
+			for (int i = 0; i < NUM_ELEMENTS && running; i++) {
+				ctx.collect(Tuple2.of(i, "message #" + i));
+			}
+		}
+
+		@Override
+		public void cancel() {
+			running = false;
+		}
+	}
+
+	private static class TestIndexRequestBuilder implements IndexRequestBuilder<Tuple2<Integer, String>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public IndexRequest createIndexRequest(Tuple2<Integer, String> element, RuntimeContext ctx) {
+			Map<String, Object> json = new HashMap<>();
+			json.put("data", element.f1);
+
+			return Requests.indexRequest()
+					.index("my-index")
+					.type("my-type")
+					.id(element.f0.toString())
+					.source(json);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..dc20726
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
@@ -0,0 +1,27 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+log4j.rootLogger=OFF, testlogger
+
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml b/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..45b3b92
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/src/test/resources/logback-test.xml
@@ -0,0 +1,30 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+    <logger name="org.apache.flink.streaming" level="WARN"/>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/pom.xml b/flink-streaming-connectors/flink-connector-filesystem/pom.xml
new file mode 100644
index 0000000..7c4dc98
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/pom.xml
@@ -0,0 +1,112 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-streaming-connectors-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-filesystem</artifactId>
+	<name>flink-connector-filesystem</name>
+
+	<packaging>jar</packaging>
+
+	<!--
+		This is a Hadoop2 only flink module.
+	-->
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>${shading-artifact.name}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-tests</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime</artifactId>
+			<scope>test</scope>
+			<type>test-jar</type>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<scope>test</scope>
+			<type>test-jar</type>
+			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>test</scope>
+			<type>test-jar</type>
+			<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
+		</dependency>
+
+	</dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
new file mode 100644
index 0000000..913da97
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Bucketer.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs;
+
+import org.apache.hadoop.fs.Path;
+
+import java.io.Serializable;
+
+/**
+ * A bucketer is used with a {@link RollingSink}
+ * to put emitted elements into rolling files.
+ *
+ * <p>
+ * The {@code RollingSink} has one active bucket that it is writing to at a time. Whenever
+ * a new element arrives it will ask the {@code Bucketer} if a new bucket should be started and
+ * the old one closed. The {@code Bucketer} can, for example, decide to start new buckets
+ * based on system time.
+ */
+public interface Bucketer extends Serializable {
+
+	/**
+	 * Returns {@code true} when a new bucket should be started.
+	 *
+	 * @param currentBucketPath The bucket {@code Path} that is currently being used.
+	 */
+	boolean shouldStartNewBucket(Path basePath, Path currentBucketPath);
+
+	/**
+	 * Returns the {@link Path} of a new bucket file.
+	 *
+	 * @param basePath The base path containing all the buckets.
+	 *
+	 * @return The complete new {@code Path} of the new bucket. This should include the {@code basePath}
+	 *      and also the {@code subtaskIndex} tp avoid clashes with parallel sinks.
+	 */
+	Path getNextBucketPath(Path basePath);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java
new file mode 100644
index 0000000..152c75a
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Clock.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.fs;
+
+/**
+ * A clock that can provide the current time.
+ *
+ * <p>
+ * Normally this would be system time, but for testing a custom {@code Clock} can be provided.
+ */
+public interface Clock {
+
+	/**
+	 * Return the current system time in milliseconds.
+	 */
+	public long currentTimeMillis();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java
new file mode 100644
index 0000000..0be40f5
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/DateTimeBucketer.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs;
+
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+/**
+ * A {@link Bucketer} that assigns to buckets based on current system time.
+ *
+ * <p>
+ * The {@code DateTimeBucketer} will create directories of the following form:
+ * {@code /{basePath}/{dateTimePath}/}. The {@code basePath} is the path
+ * that was specified as a base path when creating the
+ * {@link RollingSink}. The {@code dateTimePath}
+ * is determined based on the current system time and the user provided format string.
+ *
+ * <p>
+ * {@link SimpleDateFormat} is used to derive a date string from the current system time and
+ * the date format string. The default format string is {@code "yyyy-MM-dd--HH"} so the rolling
+ * files will have a granularity of hours.
+ *
+ *
+ * <p>
+ * Example:
+ *
+ * <pre>{@code
+ *     Bucketer buck = new DateTimeBucketer("yyyy-MM-dd--HH");
+ * }</pre>
+ *
+ * This will create for example the following bucket path:
+ * {@code /base/1976-12-31-14/}
+ *
+ */
+public class DateTimeBucketer implements Bucketer {
+
+	private static Logger LOG = LoggerFactory.getLogger(DateTimeBucketer.class);
+
+	private static final long serialVersionUID = 1L;
+
+	private static final String DEFAULT_FORMAT_STRING = "yyyy-MM-dd--HH";
+
+	// We have this so that we can manually set it for tests.
+	private static Clock clock = new SystemClock();
+
+	private final String formatString;
+
+	private transient SimpleDateFormat dateFormatter;
+
+	/**
+	 * Creates a new {@code DateTimeBucketer} with format string {@code "yyyy-MM-dd--HH"}.
+	 */
+	public DateTimeBucketer() {
+		this(DEFAULT_FORMAT_STRING);
+	}
+
+	/**
+	 * Creates a new {@code DateTimeBucketer} with the given date/time format string.
+	 *
+	 * @param formatString The format string that will be given to {@code SimpleDateFormat} to determine
+	 *                     the bucket path.
+	 */
+	public DateTimeBucketer(String formatString) {
+		this.formatString = formatString;
+
+		this.dateFormatter = new SimpleDateFormat(formatString);
+	}
+
+	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+		in.defaultReadObject();
+
+		this.dateFormatter = new SimpleDateFormat(formatString);
+	}
+
+
+	@Override
+	public boolean shouldStartNewBucket(Path basePath, Path currentBucketPath) {
+		String newDateTimeString = dateFormatter.format(new Date(clock.currentTimeMillis()));
+		return !(new Path(basePath, newDateTimeString).equals(currentBucketPath));
+	}
+
+	@Override
+	public Path getNextBucketPath(Path basePath) {
+		String newDateTimeString = dateFormatter.format(new Date(clock.currentTimeMillis()));
+		return new Path(basePath + "/" + newDateTimeString);
+	}
+
+	@Override
+	public String toString() {
+		return "DateTimeBucketer{" +
+				"formatString='" + formatString + '\'' +
+				'}';
+	}
+
+	/**
+	 * This sets the internal {@link Clock} implementation. This method should only be used for testing
+	 *
+	 * @param newClock The new clock to set.
+	 */
+	public static void setClock(Clock newClock) {
+		clock = newClock;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.java
new file mode 100644
index 0000000..1307d11
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/NonRollingBucketer.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.connectors.fs;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * A {@link org.apache.flink.streaming.connectors.fs.Bucketer} that does not perform any
+ * rolling of files. All files are written to the base path.
+ */
+public class NonRollingBucketer implements Bucketer {
+	private static final long serialVersionUID = 1L;
+
+	@Override
+	public boolean shouldStartNewBucket(Path basePath, Path currentBucketPath) {
+		return false;
+	}
+
+	@Override
+	public Path getNextBucketPath(Path basePath) {
+		return basePath;
+	}
+
+	@Override
+	public String toString() {
+		return "NonRollingBucketer";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
new file mode 100644
index 0000000..c705767
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/RollingSink.java
@@ -0,0 +1,900 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.fs;
+
+import com.google.common.collect.Lists;
+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.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Sink that emits its input elements to rolling {@link org.apache.hadoop.fs.FileSystem} files. This
+ * is itegrated with the checkpointing mechanism to provide exactly once semantics.
+ *
+ * <p>
+ * When creating the sink a {@code basePath} must be specified. The base directory contains
+ * one directory for every bucket. The bucket directories themselves contain several part files.
+ * These contain the actual written data.
+ *
+ * <p>
+ * The sink uses a {@link Bucketer} to determine the name of bucket directories inside the
+ * base directory. Whenever the {@code Bucketer} returns a different directory name than
+ * it returned before the sink will close the current part files inside that bucket
+ * and start the new bucket directory. The default bucketer is a {@link DateTimeBucketer} with
+ * date format string {@code ""yyyy-MM-dd--HH"}. You can specify a custom {@code Bucketer}
+ * using {@link #setBucketer(Bucketer)}. For example, use
+ * {@link org.apache.flink.streaming.connectors.fs.NonRollingBucketer} if you don't want to have
+ * buckets but still write part files in a fault-tolerant way.
+ *
+ * <p>
+ * The filenames of the part files contain the part prefix, the parallel subtask index of the sink
+ * and a rolling counter, for example {@code "part-1-17"}. Per default the part prefix is
+ * {@code "part"} but this can be
+ * configured using {@link #setPartPrefix(String)}. When a part file becomes bigger
+ * than the batch size the current part file is closed, the part counter is increased and
+ * a new part file is created. The batch size defaults to {@code 384MB}, this can be configured
+ * using {@link #setBatchSize(long)}.
+ *
+ * <p>
+ * Part files can be in one of three states: in-progress, pending or finished. The reason for this
+ * is how the sink works together with the checkpointing mechanism to provide exactly-once semantics
+ * and fault-tolerance. The part file that is currently being written to is in-progress. Once
+ * a part file is closed for writing it becomes pending. When a checkpoint is successful the
+ * currently pending files will be moved to finished. If a failure occurs the pending files
+ * will be deleted to reset state to the last checkpoint. The data in in-progress files will
+ * also have to be rolled back. If the {@code FileSystem} supports the {@code truncate} call
+ * this will be used to reset the file back to a previous state. If not, a special file
+ * with the same name as the part file and the suffix {@code ".valid-length"} will be written
+ * that contains the length up to which the file contains valid data. When reading the file
+ * it must be ensured that it is only read up to that point. The prefixes and suffixes for
+ * the different file states and valid-length files can be configured, for example with
+ * {@link #setPendingSuffix(String)}.
+ *
+ * <p>
+ * Note: If checkpointing is not enabled the pending files will never be moved to the finished state.
+ * In that case, the pending suffix/prefix can be set to {@code ""} to make the sink work
+ * in a non-fault-tolerant way but still provide output without prefixes and suffixes.
+ *
+ * <p>
+ * The part files are written using an instance of {@link Writer}. By default
+ * {@link org.apache.flink.streaming.connectors.fs.StringWriter} is used, which writes the result
+ * of {@code toString()} for every element. Separated by newlines. You can configure the writer
+ * using {@link #setWriter(Writer)}. For example,
+ * {@link org.apache.flink.streaming.connectors.fs.SequenceFileWriter} can be used to write
+ * Hadoop {@code SequenceFiles}.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre>{@code
+ *     new RollingSink<Tuple2<IntWritable, Text>>(outPath)
+ *         .setWriter(new SequenceFileWriter<IntWritable, Text>())
+ *         .setBucketer(new DateTimeBucketer("yyyy-MM-dd--HHmm")
+ * }</pre>
+ *
+ * This will create a sink that writes to {@code SequenceFiles} and rolls every minute.
+ *
+ * @see org.apache.flink.streaming.connectors.fs.DateTimeBucketer
+ * @see StringWriter
+ * @see SequenceFileWriter
+ *
+ * @param <T> Type of the elements emitted by this sink
+ */
+public class RollingSink<T> extends RichSinkFunction<T> implements InputTypeConfigurable, Checkpointed<RollingSink.BucketState>, CheckpointNotifier {
+	private static final long serialVersionUID = 1L;
+
+	private static Logger LOG = LoggerFactory.getLogger(RollingSink.class);
+
+
+	// --------------------------------------------------------------------------------------------
+	//  User configuration values
+	// --------------------------------------------------------------------------------------------
+	// These are initialized with some defaults but are meant to be changeable by the user
+
+	/**
+	 * The default maximum size of part files.
+	 *
+	 * 6 times the default block size
+	 */
+	private final long DEFAULT_BATCH_SIZE = 1024L * 1024L * 384L;
+
+	/**
+	 * This is used for part files that we are writing to but which where not yet confirmed
+	 * by a checkpoint.
+	 */
+	private final String DEFAULT_IN_PROGRESS_SUFFIX = ".in-progress";
+
+	/**
+	 * See above, but for prefix
+	 */
+	private final String DEFAULT_IN_PROGRESS_PREFIX = "_";
+
+	/**
+	 * This is used for part files that we are not writing to but which are not yet confirmed by
+	 * checkpoint.
+	 */
+	private final String DEFAULT_PENDING_SUFFIX = ".pending";
+
+	/**
+	 * See above, but for prefix.
+	 */
+	private final String DEFAULT_PENDING_PREFIX = "_";
+
+	/**
+	 * When truncate() is not supported on the used FileSystem we instead write a
+	 * file along the part file with this ending that contains the length up to which
+	 * the part file is valid.
+	 */
+	private final String DEFAULT_VALID_SUFFIX = ".valid-length";
+
+	/**
+	 * See above, but for prefix.
+	 */
+	private final String DEFAULT_VALID_PREFIX = "_";
+
+	/**
+	 * The default prefix for part files.
+	 */
+	private final String DEFAULT_PART_REFIX = "part";
+
+	/**
+	 * The base {@code Path} that stored all rolling bucket directories.
+	 */
+	private final String basePath;
+
+	/**
+	 * The {@code Bucketer} that is used to determine the path of bucket directories.
+	 */
+	private Bucketer bucketer;
+
+	/**
+	 * We have a template and call duplicate() for each parallel writer in open() to get the actual
+	 * writer that is used for the part files.
+	 */
+	private Writer<T> writerTemplate;
+
+	/**
+	 * The actual writer that we user for writing the part files.
+	 */
+	private Writer<T> writer;
+
+	/**
+	 * Maximum size of part files. If files exceed this we close and create a new one in the same
+	 * bucket directory.
+	 */
+	private long batchSize;
+
+	/**
+	 * If this is true we remove any leftover in-progress/pending files when the sink is opened.
+	 *
+	 * <p>
+	 * This should only be set to false if using the sink without checkpoints, to not remove
+	 * the files already in the directory.
+	 */
+	private boolean cleanupOnOpen = true;
+
+	// These are the actually configured prefixes/suffixes
+	private String inProgressSuffix = DEFAULT_IN_PROGRESS_SUFFIX;
+	private String inProgressPrefix = DEFAULT_IN_PROGRESS_PREFIX;
+
+	private String pendingSuffix = DEFAULT_PENDING_SUFFIX;
+	private String pendingPrefix = DEFAULT_PENDING_PREFIX;
+
+	private String validLengthSuffix = DEFAULT_VALID_SUFFIX;
+	private String validLengthPrefix= DEFAULT_VALID_PREFIX;
+
+	private String partPrefix = DEFAULT_PART_REFIX;
+
+	/**
+	 * The part file that we are currently writing to.
+	 */
+	private transient Path currentPartPath;
+
+	/**
+	 * The bucket directory that we are currently filling.
+	 */
+	private transient Path currentBucketDirectory;
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal fields (not configurable by user)
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * The {@code FSDataOutputStream} for the current part file.
+	 */
+	private transient FSDataOutputStream outStream;
+
+	/**
+	 * Our subtask index, retrieved from the {@code RuntimeContext} in {@link #open}.
+	 */
+	private transient int subtaskIndex;
+
+	/**
+	 * For counting the part files inside a bucket directory. Part files follow the patter
+	 * {@code "{part-prefix}-{subtask}-{count}"}. When creating new part files we increase the counter.
+	 */
+	private transient int partCounter;
+
+	/**
+	 * We use reflection to get the hflush method or use sync as a fallback.
+	 * The idea for this and the code comes from the Flume HDFS Sink.
+	 */
+	private transient Method refHflushOrSync;
+
+	/**
+	 * We use reflection to get the .truncate() method, this is only available starting with
+	 * Hadoop 2.7
+	 */
+	private transient Method refTruncate;
+
+	/**
+	 * The state object that is handled by flink from snapshot/restore. In there we store the
+	 * current part file path, the valid length of the in-progress files and pending part files.
+	 */
+	private transient BucketState bucketState;
+
+	/**
+	 * Creates a new {@code RollingSink} that writes files to the given base directory.
+	 *
+	 * <p>
+	 * This uses a{@link DateTimeBucketer} as bucketer and a {@link StringWriter} has writer.
+	 * The maximum bucket size is set to 384 MB.
+	 *
+	 * @param basePath The directory to which to write the bucket files.
+	 */
+	public RollingSink(String basePath) {
+		this.basePath = basePath;
+		this.bucketer = new DateTimeBucketer();
+		this.batchSize = DEFAULT_BATCH_SIZE;
+		this.writerTemplate = new StringWriter<>();
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
+		if (this.writerTemplate instanceof InputTypeConfigurable) {
+			((InputTypeConfigurable) writerTemplate).setInputType(type, executionConfig);
+		}
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		super.open(parameters);
+
+		subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		partCounter = 0;
+
+		this.writer = writerTemplate.duplicate();
+
+		if (bucketState == null) {
+			bucketState = new BucketState();
+		}
+
+		FileSystem fs = new Path(basePath).getFileSystem(new org.apache.hadoop.conf.Configuration());
+		refTruncate = reflectTruncate(fs);
+
+		// delete pending/in-progress files that might be left if we fail while
+		// no checkpoint has yet been done
+		try {
+			if (fs.exists(new Path(basePath)) && cleanupOnOpen) {
+				RemoteIterator<LocatedFileStatus> bucketFiles = fs.listFiles(new Path(basePath), true);
+
+				while (bucketFiles.hasNext()) {
+					LocatedFileStatus file = bucketFiles.next();
+					if (file.getPath().toString().endsWith(pendingSuffix)) {
+						// only delete files that contain our subtask index
+						if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
+							LOG.debug("Deleting leftover pending file {}", file.getPath().toString());
+							fs.delete(file.getPath(), true);
+						}
+					}
+					if (file.getPath().toString().endsWith(inProgressSuffix)) {
+						// only delete files that contain our subtask index
+						if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
+							LOG.debug("Deleting leftover in-progress file {}", file.getPath().toString());
+							fs.delete(file.getPath(), true);
+						}
+					}
+				}
+			}
+		} catch (IOException e) {
+			LOG.error("Error while deleting leftover pending/in-progress files: {}", e);
+			throw new RuntimeException("Error while deleting leftover pending/in-progress files.", e);
+		}
+	}
+
+	@Override
+	public void close() throws Exception {
+//		boolean interrupted = Thread.interrupted();
+		closeCurrentPartFile();
+
+//		if (interrupted) {
+//			Thread.currentThread().interrupt();
+//		}
+	}
+
+	@Override
+	public void invoke(T value) throws Exception {
+
+		if (shouldRoll()) {
+			openNewPartFile();
+		}
+
+		writer.write(value);
+	}
+
+	/**
+	 * Determines whether we should change the bucket file we are writing to.
+	 *
+	 * <p>
+	 * This will roll if no file was created yet, if the file size is larger than the specified size
+	 * or if the {@code Bucketer} determines that we should roll.
+	 */
+	private boolean shouldRoll() throws IOException {
+		boolean shouldRoll = false;
+		if (outStream == null) {
+			shouldRoll = true;
+			LOG.debug("RollingSink {} starting new initial bucket. ", subtaskIndex);
+		}
+		if (bucketer.shouldStartNewBucket(new Path(basePath), currentBucketDirectory)) {
+			shouldRoll = true;
+			LOG.debug("RollingSink {} starting new bucket because {} said we should. ", subtaskIndex, bucketer);
+			// we will retrieve a new bucket base path in openNewPartFile so reset the part counter
+			partCounter = 0;
+		}
+		if (outStream != null) {
+			long writePosition = outStream.getPos();
+			if (outStream != null && writePosition > batchSize) {
+				shouldRoll = true;
+				LOG.debug(
+						"RollingSink {} starting new bucket because file position {} is above batch size {}.",
+						subtaskIndex,
+						writePosition,
+						batchSize);
+			}
+		}
+		return shouldRoll;
+	}
+
+	/**
+	 * Opens a new part file.
+	 *
+	 * <p>
+	 * This closes the old bucket file and retrieves a new bucket path from the {@code Bucketer}.
+	 */
+	private void openNewPartFile() throws Exception {
+		closeCurrentPartFile();
+
+		org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration();
+
+		FileSystem fs = new Path(basePath).getFileSystem(conf);
+
+		Path newBucketDirectory = bucketer.getNextBucketPath(new Path(basePath));
+
+		if (!newBucketDirectory.equals(currentBucketDirectory)) {
+			currentBucketDirectory = newBucketDirectory;
+			try {
+				if (fs.mkdirs(currentBucketDirectory)) {
+					LOG.debug("Created new bucket directory: {}", currentBucketDirectory);
+				}
+			} catch (IOException e) {
+				throw new RuntimeException("Could not create base path for new rolling file.", e);
+			}
+		}
+
+
+		currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
+
+		// This should work since there is only one parallel subtask that tries names with
+		// our subtask id. Otherwise we would run into concurrency issues here.
+		while (fs.exists(currentPartPath) || fs.exists(new Path(currentPartPath.getParent(), pendingPrefix + currentPartPath.getName()).suffix(pendingSuffix))) {
+			partCounter++;
+			currentPartPath = new Path(currentBucketDirectory, partPrefix + "-" + subtaskIndex + "-" + partCounter);
+		}
+
+		// increase, so we don't have to check for this name next time
+		partCounter++;
+
+		LOG.debug("Next part path is {}", currentPartPath.toString());
+
+		Path inProgressPath = new Path(currentPartPath.getParent(), inProgressPrefix + currentPartPath.getName()).suffix(inProgressSuffix);
+
+
+
+		outStream = fs.create(inProgressPath, false);
+
+		// We do the reflection here since this is the first time that we have a FSDataOutputStream
+		if (refHflushOrSync == null) {
+			refHflushOrSync = reflectHflushOrSync(outStream);
+		}
+
+		writer.open(outStream);
+	}
+
+	/**
+	 * Closes the current part file.
+	 *
+	 * <p>
+	 * This moves the current in-progress part file to a pending file and adds it to the list
+	 * of pending files in our bucket state.
+	 */
+	private void closeCurrentPartFile() throws Exception {
+		if (writer != null) {
+			writer.close();
+		}
+
+		if (outStream != null) {
+			hflushOrSync(outStream);
+			outStream.close();
+			outStream = null;
+		}
+		if (currentPartPath != null) {
+			Path inProgressPath = new Path(currentPartPath.getParent(), inProgressPrefix + currentPartPath.getName()).suffix(inProgressSuffix);
+			Path pendingPath = new Path(currentPartPath.getParent(), pendingPrefix + currentPartPath.getName()).suffix(pendingSuffix);
+			FileSystem fs = inProgressPath.getFileSystem(new org.apache.hadoop.conf.Configuration());
+			fs.rename(inProgressPath, pendingPath);
+			LOG.debug("Moving in-progress bucket {} to pending file {}",
+					inProgressPath,
+					pendingPath);
+			this.bucketState.pendingFiles.add(currentPartPath.toString());
+		}
+	}
+
+	/**
+	 * If hflush is available in this version of HDFS, then this method calls
+	 * hflush, else it calls sync.
+	 * @param os - The stream to flush/sync
+	 * @throws java.io.IOException
+	 *
+	 * <p>
+	 * Note: This code comes from Flume
+	 */
+	protected void hflushOrSync(FSDataOutputStream os) throws IOException {
+		try {
+			// At this point the refHflushOrSync cannot be null,
+			// since register method would have thrown if it was.
+			this.refHflushOrSync.invoke(os);
+		} catch (InvocationTargetException e) {
+			String msg = "Error while trying to hflushOrSync!";
+			LOG.error(msg + " " + e.getCause());
+			Throwable cause = e.getCause();
+			if(cause != null && cause instanceof IOException) {
+				throw (IOException)cause;
+			}
+			throw new RuntimeException(msg, e);
+		} catch (Exception e) {
+			String msg = "Error while trying to hflushOrSync!";
+			LOG.error(msg + " " + e);
+			throw new RuntimeException(msg, e);
+		}
+	}
+
+	/**
+	 * Gets the hflush call using reflection. Fallback to sync if hflush is not available.
+	 *
+	 * <p>
+	 * Note: This code comes from Flume
+	 */
+	private Method reflectHflushOrSync(FSDataOutputStream os) {
+		Method m = null;
+		if(os != null) {
+			Class<?> fsDataOutputStreamClass = os.getClass();
+			try {
+				m = fsDataOutputStreamClass.getMethod("hflush");
+			} catch (NoSuchMethodException ex) {
+				LOG.debug("HFlush not found. Will use sync() instead");
+				try {
+					m = fsDataOutputStreamClass.getMethod("sync");
+				} catch (Exception ex1) {
+					String msg = "Neither hflush not sync were found. That seems to be " +
+							"a problem!";
+					LOG.error(msg);
+					throw new RuntimeException(msg, ex1);
+				}
+			}
+		}
+		return m;
+	}
+
+	/**
+	 * Gets the truncate() call using reflection.
+	 *
+	 * <p>
+	 * Note: This code comes from Flume
+	 */
+	private Method reflectTruncate(FileSystem fs) {
+		Method m = null;
+		if(fs != null) {
+			Class<?> fsClass = fs.getClass();
+			try {
+				m = fsClass.getMethod("truncate", Path.class, long.class);
+			} catch (NoSuchMethodException ex) {
+				LOG.debug("Truncate not found. Will write a file with suffix '{}' " +
+						" and prefix '{}' to specify how many bytes in a bucket are valid.", validLengthSuffix, validLengthPrefix);
+				return null;
+			}
+
+
+			// verify that truncate actually works
+			FSDataOutputStream outputStream;
+			Path testPath = new Path(UUID.randomUUID().toString());
+			try {
+				outputStream = fs.create(testPath);
+				outputStream.writeUTF("hello");
+				outputStream.close();
+			} catch (IOException e) {
+				LOG.error("Could not create file for checking if truncate works.", e);
+				throw new RuntimeException("Could not create file for checking if truncate works.", e);
+			}
+
+
+			try {
+				m.invoke(fs, testPath, 2);
+			} catch (IllegalAccessException | InvocationTargetException e) {
+				LOG.debug("Truncate is not supported.", e);
+				m = null;
+			}
+
+			try {
+				fs.delete(testPath, false);
+			} catch (IOException e) {
+				LOG.error("Could not delete truncate test file.", e);
+				throw new RuntimeException("Could not delete truncate test file.", e);
+			}
+		}
+		return m;
+	}
+
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		synchronized (bucketState.pendingFilesPerCheckpoint) {
+			Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
+			Set<Long> checkpointsToRemove = Sets.newHashSet();
+			for (Long pastCheckpointId : pastCheckpointIds) {
+				if (pastCheckpointId <= checkpointId) {
+					LOG.debug("Moving pending files to final location for checkpoint {}", pastCheckpointId);
+					// All the pending files are buckets that have been completed but are waiting to be renamed
+					// to their final name
+					for (String filename : bucketState.pendingFilesPerCheckpoint.get(
+							pastCheckpointId)) {
+						Path finalPath = new Path(filename);
+						Path pendingPath = new Path(finalPath.getParent(),
+								pendingPrefix + finalPath.getName()).suffix(pendingSuffix);
+
+						FileSystem fs = pendingPath.getFileSystem(new org.apache.hadoop.conf.Configuration());
+						fs.rename(pendingPath, finalPath);
+						LOG.debug(
+								"Moving pending file {} to final location after complete checkpoint {}.",
+								pendingPath,
+								pastCheckpointId);
+					}
+					checkpointsToRemove.add(pastCheckpointId);
+				}
+			}
+			for (Long toRemove: checkpointsToRemove) {
+				bucketState.pendingFilesPerCheckpoint.remove(toRemove);
+			}
+		}
+	}
+
+	@Override
+	public BucketState snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+		if (writer != null) {
+			writer.flush();
+		}
+		if (outStream != null) {
+			hflushOrSync(outStream);
+			bucketState.currentFile = currentPartPath.toString();
+			bucketState.currentFileValidLength = outStream.getPos();
+		}
+		synchronized (bucketState.pendingFilesPerCheckpoint) {
+			bucketState.pendingFilesPerCheckpoint.put(checkpointId, bucketState.pendingFiles);
+		}
+		bucketState.pendingFiles = Lists.newArrayList();
+		return bucketState;
+	}
+
+	@Override
+	public void restoreState(BucketState state) {
+		bucketState = state;
+		// we can clean all the pending files since they where renamed to final files
+		// after this checkpoint was successfull
+		bucketState.pendingFiles.clear();
+		FileSystem fs = null;
+		try {
+			fs = new Path(basePath).getFileSystem(new org.apache.hadoop.conf.Configuration());
+		} catch (IOException e) {
+			LOG.error("Error while creating FileSystem in checkpoint restore.", e);
+			throw new RuntimeException("Error while creating FileSystem in checkpoint restore.", e);
+		}
+		if (bucketState.currentFile != null) {
+			// We were writing to a file when the last checkpoint occured. This file can either
+			// be still in-progress or became a pending file at some point after the checkpoint.
+			// Either way, we have to truncate it back to a valid state (or write a .valid-length)
+			// file that specifies up to which length it is valid and rename it to the final name
+			// before starting a new bucket file.
+			Path partPath = new Path(bucketState.currentFile);
+			try {
+				Path partPendingPath = new Path(partPath.getParent(), pendingPrefix + partPath.getName()).suffix(
+						pendingSuffix);
+				Path partInProgressPath = new Path(partPath.getParent(), inProgressPrefix + partPath.getName()).suffix(inProgressSuffix);
+
+				if (fs.exists(partPendingPath)) {
+					LOG.debug("In-progress file {} has been moved to pending after checkpoint, moving to final location.", partPath);
+					// has been moved to pending in the mean time, rename to final location
+					fs.rename(partPendingPath, partPath);
+				} else if (fs.exists(partInProgressPath)) {
+					LOG.debug("In-progress file {} is still in-progress, moving to final location.", partPath);
+					// it was still in progress, rename to final path
+					fs.rename(partInProgressPath, partPath);
+				} else {
+					LOG.error("In-Progress file {} was neither moved to pending nor is still in progress.", bucketState.currentFile);
+					throw new RuntimeException("In-Progress file " + bucketState.currentFile+ " " +
+							"was neither moved to pending nor is still in progress.");
+				}
+
+				refTruncate = reflectTruncate(fs);
+				// truncate it or write a ".valid-length" file to specify up to which point it is valid
+				if (refTruncate != null) {
+					LOG.debug("Truncating {} to valid length {}", partPath, bucketState.currentFileValidLength);
+					refTruncate.invoke(fs, partPath, bucketState.currentFileValidLength);
+				} else {
+					LOG.debug("Writing valid-length file for {} to specify valid length {}", partPath, bucketState.currentFileValidLength);
+					Path validLengthFilePath = new Path(partPath.getParent(), validLengthPrefix + partPath.getName()).suffix(validLengthSuffix);
+					FSDataOutputStream lengthFileOut = fs.create(validLengthFilePath);
+					lengthFileOut.writeUTF(Long.toString(bucketState.currentFileValidLength));
+					lengthFileOut.close();
+				}
+
+				// invalidate in the state object
+				bucketState.currentFile = null;
+				bucketState.currentFileValidLength = -1;
+			} catch (IOException e) {
+				LOG.error("Error while restoring RollingSink state.", e);
+				throw new RuntimeException("Error while restoring RollingSink state.", e);
+			} catch (InvocationTargetException | IllegalAccessException e) {
+				LOG.error("Cound not invoke truncate.", e);
+				throw new RuntimeException("Could not invoke truncate.", e);
+			}
+		}
+
+		LOG.debug("Clearing pending/in-progress files.");
+
+		// Move files that are confirmed by a checkpoint but did not get moved to final location
+		// because the checkpoint notification did not happen before a failure
+
+		Set<Long> pastCheckpointIds = bucketState.pendingFilesPerCheckpoint.keySet();
+		LOG.debug("Moving pending files to final location on restore.");
+		for (Long pastCheckpointId : pastCheckpointIds) {
+			// All the pending files are buckets that have been completed but are waiting to be renamed
+			// to their final name
+			for (String filename : bucketState.pendingFilesPerCheckpoint.get(pastCheckpointId)) {
+				Path finalPath = new Path(filename);
+				Path pendingPath = new Path(finalPath.getParent(), pendingPrefix + finalPath.getName()).suffix(pendingSuffix);
+
+				try {
+					if (fs.exists(pendingPath)) {
+						LOG.debug(
+								"Moving pending file {} to final location after complete checkpoint {}.",
+								pendingPath,
+								pastCheckpointId);
+						fs.rename(pendingPath, finalPath);
+					}
+				} catch (IOException e) {
+					LOG.error("Error while renaming pending file {} to final path {}: {}", pendingPath, finalPath, e);
+					throw new RuntimeException("Error while renaming pending file " + pendingPath+ " to final path " + finalPath, e);
+				}
+			}
+		}
+		bucketState.pendingFiles.clear();
+		synchronized (bucketState.pendingFilesPerCheckpoint) {
+			bucketState.pendingFilesPerCheckpoint.clear();
+		}
+
+		// we need to get this here since open() has not yet been called
+		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
+		// delete pending files
+		try {
+
+			RemoteIterator<LocatedFileStatus> bucketFiles = fs.listFiles(new Path(basePath), true);
+
+			while (bucketFiles.hasNext()) {
+				LocatedFileStatus file = bucketFiles.next();
+				if (file.getPath().toString().endsWith(pendingSuffix)) {
+					// only delete files that contain our subtask index
+					if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
+						fs.delete(file.getPath(), true);
+					}
+				}
+				if (file.getPath().toString().endsWith(inProgressSuffix)) {
+					// only delete files that contain our subtask index
+					if (file.getPath().toString().contains(partPrefix + "-" + subtaskIndex)) {
+						LOG.debug("Deleting in-progress file {}", file.getPath().toString());
+						fs.delete(file.getPath(), true);
+					}
+				}
+			}
+		} catch (IOException e) {
+			LOG.error("Error while deleting old pending files: {}", e);
+			throw new RuntimeException("Error while deleting old pending files.", e);
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Setters for User configuration values
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Sets the maximum bucket size in bytes.
+	 *
+	 * <p>
+	 * When a bucket part file becomes larger than this size a new bucket part file is started and
+	 * the old one is closed. The name of the bucket files depends on the {@link Bucketer}.
+	 *
+	 * @param batchSize The bucket part file size in bytes.
+	 */
+	public RollingSink<T> setBatchSize(long batchSize) {
+		this.batchSize = batchSize;
+		return this;
+	}
+
+	/**
+	 * Sets the {@link Bucketer} to use for determining the bucket files to write to.
+	 *
+	 * @param bucketer The bucketer to use.
+	 */
+	public RollingSink<T> setBucketer(Bucketer bucketer) {
+		this.bucketer = bucketer;
+		return this;
+	}
+
+	/**
+	 * Sets the {@link Writer} to be used for writing the incoming elements to bucket files.
+	 *
+	 * @param writer The {@code Writer} to use.
+	 */
+	public RollingSink<T> setWriter(Writer<T> writer) {
+		this.writerTemplate = writer;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of in-progress part files. The default is {@code "in-progress"}.
+	 */
+	public RollingSink<T> setInProgressSuffix(String inProgressSuffix) {
+		this.inProgressSuffix = inProgressSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of in-progress part files. The default is {@code "_"}.
+	 */
+	public RollingSink<T> setInProgressPrefix(String inProgressPrefix) {
+		this.inProgressPrefix = inProgressPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of pending part files. The default is {@code ".pending"}.
+	 */
+	public RollingSink<T> setPendingSuffix(String pendingSuffix) {
+		this.pendingSuffix = pendingSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of pending part files. The default is {@code "_"}.
+	 */
+	public RollingSink<T> setPendingPrefix(String pendingPrefix) {
+		this.pendingPrefix = pendingPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the suffix of valid-length files. The default is {@code ".valid-length"}.
+	 */
+	public RollingSink<T> setValidLengthSuffix(String validLengthSuffix) {
+		this.validLengthSuffix = validLengthSuffix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of valid-length files. The default is {@code "_"}.
+	 */
+	public RollingSink<T> setValidLengthPrefix(String validLengthPrefix) {
+		this.validLengthPrefix = validLengthPrefix;
+		return this;
+	}
+
+	/**
+	 * Sets the prefix of part files.  The default is {@code "part"}.
+	 */
+	public RollingSink<T> setPartPrefix(String partPrefix) {
+		this.partPrefix = partPrefix;
+		return this;
+	}
+
+	/**
+	 * Disable cleanup of leftover in-progress/pending files when the sink is opened.
+	 *
+	 * <p>
+	 * This should only be disabled if using the sink without checkpoints, to not remove
+	 * the files already in the directory.
+	 */
+	public RollingSink<T> disableCleanupOnOpen() {
+		this.cleanupOnOpen = false;
+		return this;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Internal Classes
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * This is used for keeping track of the current in-progress files and files that we mark
+	 * for moving from pending to final location after we get a checkpoint-complete notification.
+	 */
+	static final class BucketState implements Serializable {
+		private static final long serialVersionUID = 1L;
+
+		/**
+		 * The file that was in-progress when the last checkpoint occured.
+		 */
+		String currentFile = null;
+
+		/**
+		 * The valid length of the in-progress file at the time of the last checkpoint.
+		 */
+		long currentFileValidLength = -1;
+
+		/**
+		 * Pending files that accumulated since the last checkpoint.
+		 */
+		List<String> pendingFiles = Lists.newArrayList();
+
+		/**
+		 * When doing a checkpoint we move the pending files since the last checkpoint to this map
+		 * with the id of the checkpoint. When we get the checkpoint-complete notification we move
+		 * pending files of completed checkpoints to their final location.
+		 */
+		final Map<Long, List<String>> pendingFilesPerCheckpoint = Maps.newHashMap();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
new file mode 100644
index 0000000..928d96e
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SequenceFileWriter.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs;
+
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+
+import java.io.IOException;
+
+/**
+ * A {@link Writer} that writes the bucket files as Hadoop {@link SequenceFile SequenceFiles}.
+ * The input to the {@link RollingSink} must
+ * be a {@link org.apache.flink.api.java.tuple.Tuple2} of two Hadopo
+ * {@link org.apache.hadoop.io.Writable Writables}.
+ *
+ * @param <K> The type of the first tuple field.
+ * @param <V> The type of the second tuple field.
+ */
+public class SequenceFileWriter<K extends Writable, V extends Writable> implements Writer<Tuple2<K, V>>, InputTypeConfigurable {
+	private static final long serialVersionUID = 1L;
+
+	private final String compressionCodecName;
+
+	private SequenceFile.CompressionType compressionType;
+
+	private transient FSDataOutputStream outputStream;
+
+	private transient SequenceFile.Writer writer;
+
+	private Class<K> keyClass;
+
+	private Class<V> valueClass;
+
+	/**
+	 * Creates a new {@code SequenceFileWriter} that writes sequence files without compression.
+	 */
+	public SequenceFileWriter() {
+		this("None", SequenceFile.CompressionType.NONE);
+	}
+
+	/**
+	 * Creates a new {@code SequenceFileWriter} that writes sequence with the given
+	 * compression codec and compression type.
+	 *
+	 * @param compressionCodecName Name of a Hadoop Compression Codec.
+	 * @param compressionType The compression type to use.
+	 */
+	public SequenceFileWriter(String compressionCodecName,
+			SequenceFile.CompressionType compressionType) {
+		this.compressionCodecName = compressionCodecName;
+		this.compressionType = compressionType;
+	}
+
+	@Override
+	public void open(FSDataOutputStream outStream) throws IOException {
+		if (outputStream != null) {
+			throw new IllegalStateException("SequenceFileWriter has already been opened.");
+		}
+		if (keyClass == null) {
+			throw new IllegalStateException("Key Class has not been initialized.");
+		}
+		if (valueClass == null) {
+			throw new IllegalStateException("Value Class has not been initialized.");
+		}
+
+		this.outputStream = outStream;
+
+		CompressionCodec codec = null;
+
+		if (!compressionCodecName.equals("None")) {
+			CompressionCodecFactory codecFactory = new CompressionCodecFactory(new Configuration());
+			codec = codecFactory.getCodecByName(compressionCodecName);
+			if (codec == null) {
+				throw new RuntimeException("Codec " + compressionCodecName + " not found.");
+			}
+		}
+
+		// the non-deprecated constructor syntax is only available in recent hadoop versions...
+		writer = SequenceFile.createWriter(new Configuration(),
+				outStream,
+				keyClass,
+				valueClass,
+				compressionType,
+				codec);
+	}
+
+	@Override
+	public void flush() throws IOException {
+	}
+
+	@Override
+	public void close() throws IOException {
+		if (writer != null) {
+			writer.close();
+		}
+		writer = null;
+		outputStream = null;
+	}
+
+	@Override
+	public void write(Tuple2<K, V> element) throws IOException {
+		if (outputStream == null) {
+			throw new IllegalStateException("SequenceFileWriter has not been opened.");
+		}
+		writer.append(element.f0, element.f1);
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
+		if (!type.isTupleType()) {
+			throw new IllegalArgumentException("Input TypeInformation is not a tuple type.");
+		}
+
+		TupleTypeInfoBase<?> tupleType = (TupleTypeInfoBase<?>) type;
+
+		if (tupleType.getArity() != 2) {
+			throw new IllegalArgumentException("Input TypeInformation must be a Tuple2 type.");
+		}
+
+		TypeInformation<K> keyType = tupleType.getTypeAt(0);
+		TypeInformation<V> valueType = tupleType.getTypeAt(1);
+
+		this.keyClass = keyType.getTypeClass();
+		this.valueClass = valueType.getTypeClass();
+	}
+
+	@Override
+	public Writer<Tuple2<K, V>> duplicate() {
+		SequenceFileWriter<K, V> result = new SequenceFileWriter<>(compressionCodecName, compressionType);
+		result.keyClass = keyClass;
+		result.valueClass = valueClass;
+		return result;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java
new file mode 100644
index 0000000..ad0ab46
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/StringWriter.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs;
+
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.IllegalCharsetNameException;
+import java.nio.charset.UnsupportedCharsetException;
+
+/**
+ * A {@link Writer} that uses {@code toString()} on the input elements and writes them to
+ * the output bucket file separated by newline.
+ *
+ * @param <T> The type of the elements that are being written by the sink.
+ */
+public class StringWriter<T> implements Writer<T> {
+	private static final long serialVersionUID = 1L;
+
+	private transient FSDataOutputStream outputStream;
+
+	private String charsetName;
+
+	private transient Charset charset;
+
+	/**
+	 * Creates a new {@code StringWriter} that uses {@code "UTF-8"} charset to convert
+	 * strings to bytes.
+	 */
+	public StringWriter() {
+		this("UTF-8");
+	}
+
+	/**
+	 * Creates a new {@code StringWriter} that uses the given charset to convert
+	 * strings to bytes.
+	 *
+	 * @param charsetName Name of the charset to be used, must be valid input for {@code Charset.forName(charsetName)}
+	 */
+	public StringWriter(String charsetName) {
+		this.charsetName = charsetName;
+	}
+
+	@Override
+	public void open(FSDataOutputStream outStream) throws IOException {
+		if (outputStream != null) {
+			throw new IllegalStateException("StringWriter has already been opened.");
+		}
+		this.outputStream = outStream;
+
+		try {
+			this.charset = Charset.forName(charsetName);
+		}
+		catch (IllegalCharsetNameException e) {
+			throw new IOException("The charset " + charsetName + " is not valid.", e);
+		}
+		catch (UnsupportedCharsetException e) {
+			throw new IOException("The charset " + charsetName + " is not supported.", e);
+		}
+	}
+
+	@Override
+	public void flush() throws IOException {
+
+	}
+
+	@Override
+	public void close() throws IOException {
+		outputStream = null;
+	}
+
+	@Override
+	public void write(T element) throws IOException {
+		if (outputStream == null) {
+			throw new IllegalStateException("StringWriter has not been opened.");
+		}
+		outputStream.write(element.toString().getBytes(charset));
+		outputStream.write('\n');
+
+	}
+
+	@Override
+	public Writer<T> duplicate() {
+		return new StringWriter<>();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java
new file mode 100644
index 0000000..2bab8cf
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/SystemClock.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.fs;
+
+/**
+ * A {@link Clock} that uses {@code System.currentTimeMillis()} to determine the system time.
+ */
+public class SystemClock implements Clock {
+	@Override
+	public long currentTimeMillis() {
+		return System.currentTimeMillis();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
new file mode 100644
index 0000000..98cad32
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/Writer.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.connectors.fs;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * An implementation of {@code Writer} is used in conjunction with a
+ * {@link RollingSink} to perform the actual
+ * writing to the bucket files.
+ *
+ * @param <T> The type of the elements that are being written by the sink.
+ */
+public interface Writer<T> extends Serializable {
+
+	/**
+	 * Initializes the {@code Writer} for a newly opened bucket file.
+	 * Any internal per-bucket initialization should be performed here.
+	 *
+	 * @param outStream The {@link org.apache.hadoop.fs.FSDataOutputStream} for the newly opened file.
+	 */
+	void open(FSDataOutputStream outStream) throws IOException;
+
+	/**
+	 * Flushes out any internally held data.
+	 */
+	void flush()throws IOException ;
+
+	/**
+	 * Closes the {@code Writer}. This must not close the {@code FSDataOutputStream} that
+	 * was handed in in the {@link #open} method. Only internally held state should be
+	 * closed.
+	 */
+	void close() throws IOException ;
+
+	/**
+	 * Writes one element to the bucket file.
+	 */
+	void write(T element)throws IOException;
+
+	/**
+	 * Duplicates the {@code Writer}. This is used to get one {@code Writer} for each
+	 * parallel instance of the sink.
+	 */
+	Writer<T> duplicate();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties b/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties
new file mode 100644
index 0000000..fe60d94
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/main/resources/log4j.properties
@@ -0,0 +1,27 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+log4j.rootLogger=OFF, testlogger
+
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java
new file mode 100644
index 0000000..7d127ff
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java
@@ -0,0 +1,289 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS 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.connectors.fs;
+
+import com.google.common.collect.Sets;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.test.checkpointing.StreamFaultToleranceTestBase;
+import org.apache.flink.util.NetUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.Random;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+* Tests for {@link RollingSink}.
+*
+* <p>
+* This test only verifies the exactly once behaviour of the sink. Another test tests the
+* rolling behaviour.
+*
+* <p>
+* This differs from RollingSinkFaultToleranceITCase in that the checkpoint interval is extremely
+* high. This provokes the case that the sink restarts without any checkpoint having been performed.
+* This tests the initial cleanup of pending/in-progress files.
+*/
+public class RollingSinkFaultTolerance2ITCase extends StreamFaultToleranceTestBase {
+
+	final long NUM_STRINGS = 16_000;
+
+	@ClassRule
+	public static TemporaryFolder tempFolder = new TemporaryFolder();
+
+	private static MiniDFSCluster hdfsCluster;
+	private static org.apache.hadoop.fs.FileSystem dfs;
+
+	private static String outPath;
+
+
+
+	@BeforeClass
+	public static void createHDFS() throws IOException {
+		Configuration conf = new Configuration();
+
+		File dataDir = tempFolder.newFolder();
+
+		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
+		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+		hdfsCluster = builder.build();
+
+		dfs = hdfsCluster.getFileSystem();
+
+		outPath = "hdfs://"
+				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(),  hdfsCluster.getNameNodePort())
+				+ "/string-non-rolling-out-no-checkpoint";
+	}
+
+	@AfterClass
+	public static void destroyHDFS() {
+		if (hdfsCluster != null) {
+			hdfsCluster.shutdown();
+		}
+	}
+
+
+	@Override
+	public void testProgram(StreamExecutionEnvironment env) {
+		assertTrue("Broken test setup", NUM_STRINGS % 40 == 0);
+
+		int PARALLELISM = 6;
+
+		env.enableCheckpointing(Long.MAX_VALUE);
+		env.setParallelism(PARALLELISM);
+		env.disableOperatorChaining();
+
+		DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)).startNewChain();
+
+		DataStream<String> mapped = stream
+				.map(new OnceFailingIdentityMapper(NUM_STRINGS));
+
+		RollingSink<String> sink = new RollingSink<String>(outPath)
+				.setBucketer(new NonRollingBucketer())
+				.setBatchSize(5000)
+				.setValidLengthPrefix("")
+				.setPendingPrefix("");
+
+		mapped.addSink(sink);
+
+	}
+
+	@Override
+	public void postSubmit() throws Exception {
+		// We read the files and verify that we have read all the strings. If a valid-length
+		// file exists we only read the file to that point. (This test should work with
+		// FileSystems that support truncate() and with others as well.)
+
+		Pattern messageRegex = Pattern.compile("message (\\d*)");
+
+		// Keep a set of the message IDs that we read. The size must equal the read count and
+		// the NUM_STRINGS. If numRead is bigger than the size of the set we have seen some
+		// elements twice.
+		Set<Integer> readNumbers = Sets.newHashSet();
+		int numRead = 0;
+
+		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(
+				outPath), true);
+
+		while (files.hasNext()) {
+			LocatedFileStatus file = files.next();
+
+			if (!file.getPath().toString().endsWith(".valid-length")) {
+				int validLength = (int) file.getLen();
+				if (dfs.exists(file.getPath().suffix(".valid-length"))) {
+					FSDataInputStream inStream = dfs.open(file.getPath().suffix(".valid-length"));
+					String validLengthString = inStream.readUTF();
+					validLength = Integer.parseInt(validLengthString);
+					System.out.println("VALID LENGTH: " + validLength);
+				}
+				FSDataInputStream inStream = dfs.open(file.getPath());
+				byte[] buffer = new byte[validLength];
+				inStream.readFully(0, buffer, 0, validLength);
+				inStream.close();
+
+				ByteArrayInputStream bais = new ByteArrayInputStream(buffer);
+
+				InputStreamReader inStreamReader = new InputStreamReader(bais);
+				BufferedReader br = new BufferedReader(inStreamReader);
+
+				String line = br.readLine();
+				while (line != null) {
+					Matcher matcher = messageRegex.matcher(line);
+					if (matcher.matches()) {
+						numRead++;
+						int messageId = Integer.parseInt(matcher.group(1));
+						readNumbers.add(messageId);
+					} else {
+						Assert.fail("Read line does not match expected pattern.");
+					}
+					line = br.readLine();
+				}
+				br.close();
+				inStreamReader.close();
+				bais.close();
+			}
+		}
+
+		// Verify that we read all strings (at-least-once)
+		Assert.assertEquals(NUM_STRINGS, readNumbers.size());
+
+		// Verify that we don't have duplicates (boom!, exactly-once)
+		Assert.assertEquals(NUM_STRINGS, numRead);
+	}
+
+	private static class OnceFailingIdentityMapper extends RichMapFunction<String, String> {
+		private static final long serialVersionUID = 1L;
+
+		private static volatile boolean hasFailed = false;
+
+		private final long numElements;
+
+		private long failurePos;
+		private long count;
+
+
+		OnceFailingIdentityMapper(long numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void open(org.apache.flink.configuration.Configuration parameters) throws IOException {
+			long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
+			long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
+
+			failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
+			count = 0;
+		}
+
+		@Override
+		public String map(String value) throws Exception {
+			count++;
+			if (!hasFailed && count >= failurePos) {
+				hasFailed = true;
+				throw new Exception("Test Failure");
+			}
+
+			return value;
+		}
+	}
+
+	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
+			implements CheckpointedAsynchronously<Integer> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final long numElements;
+
+		private int index;
+
+		private volatile boolean isRunning = true;
+
+
+		StringGeneratingSourceFunction(long numElements) {
+			this.numElements = numElements;
+		}
+
+		@Override
+		public void run(SourceContext<String> ctx) throws Exception {
+			final Object lockingObject = ctx.getCheckpointLock();
+
+			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
+
+			if (index == 0) {
+				index = getRuntimeContext().getIndexOfThisSubtask();
+			}
+
+			while (isRunning && index < numElements) {
+
+				Thread.sleep(1);
+				synchronized (lockingObject) {
+					ctx.collect("message " + index);
+					index += step;
+				}
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+
+		private static String randomString(StringBuilder bld, Random rnd) {
+			final int len = rnd.nextInt(10) + 5;
+
+			for (int i = 0; i < len; i++) {
+				char next = (char) (rnd.nextInt(20000) + 33);
+				bld.append(next);
+			}
+
+			return bld.toString();
+		}
+
+		@Override
+		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+			return index;
+		}
+
+		@Override
+		public void restoreState(Integer state) {
+			index = state;
+		}
+	}
+}


[49/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java
deleted file mode 100644
index 7d127ff..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultTolerance2ITCase.java
+++ /dev/null
@@ -1,289 +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.connectors.fs;
-
-import com.google.common.collect.Sets;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.test.checkpointing.StreamFaultToleranceTestBase;
-import org.apache.flink.util.NetUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.Random;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import static org.junit.Assert.assertTrue;
-
-/**
-* Tests for {@link RollingSink}.
-*
-* <p>
-* This test only verifies the exactly once behaviour of the sink. Another test tests the
-* rolling behaviour.
-*
-* <p>
-* This differs from RollingSinkFaultToleranceITCase in that the checkpoint interval is extremely
-* high. This provokes the case that the sink restarts without any checkpoint having been performed.
-* This tests the initial cleanup of pending/in-progress files.
-*/
-public class RollingSinkFaultTolerance2ITCase extends StreamFaultToleranceTestBase {
-
-	final long NUM_STRINGS = 16_000;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	private static MiniDFSCluster hdfsCluster;
-	private static org.apache.hadoop.fs.FileSystem dfs;
-
-	private static String outPath;
-
-
-
-	@BeforeClass
-	public static void createHDFS() throws IOException {
-		Configuration conf = new Configuration();
-
-		File dataDir = tempFolder.newFolder();
-
-		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
-		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-		hdfsCluster = builder.build();
-
-		dfs = hdfsCluster.getFileSystem();
-
-		outPath = "hdfs://"
-				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(),  hdfsCluster.getNameNodePort())
-				+ "/string-non-rolling-out-no-checkpoint";
-	}
-
-	@AfterClass
-	public static void destroyHDFS() {
-		if (hdfsCluster != null) {
-			hdfsCluster.shutdown();
-		}
-	}
-
-
-	@Override
-	public void testProgram(StreamExecutionEnvironment env) {
-		assertTrue("Broken test setup", NUM_STRINGS % 40 == 0);
-
-		int PARALLELISM = 6;
-
-		env.enableCheckpointing(Long.MAX_VALUE);
-		env.setParallelism(PARALLELISM);
-		env.disableOperatorChaining();
-
-		DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)).startNewChain();
-
-		DataStream<String> mapped = stream
-				.map(new OnceFailingIdentityMapper(NUM_STRINGS));
-
-		RollingSink<String> sink = new RollingSink<String>(outPath)
-				.setBucketer(new NonRollingBucketer())
-				.setBatchSize(5000)
-				.setValidLengthPrefix("")
-				.setPendingPrefix("");
-
-		mapped.addSink(sink);
-
-	}
-
-	@Override
-	public void postSubmit() throws Exception {
-		// We read the files and verify that we have read all the strings. If a valid-length
-		// file exists we only read the file to that point. (This test should work with
-		// FileSystems that support truncate() and with others as well.)
-
-		Pattern messageRegex = Pattern.compile("message (\\d*)");
-
-		// Keep a set of the message IDs that we read. The size must equal the read count and
-		// the NUM_STRINGS. If numRead is bigger than the size of the set we have seen some
-		// elements twice.
-		Set<Integer> readNumbers = Sets.newHashSet();
-		int numRead = 0;
-
-		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(
-				outPath), true);
-
-		while (files.hasNext()) {
-			LocatedFileStatus file = files.next();
-
-			if (!file.getPath().toString().endsWith(".valid-length")) {
-				int validLength = (int) file.getLen();
-				if (dfs.exists(file.getPath().suffix(".valid-length"))) {
-					FSDataInputStream inStream = dfs.open(file.getPath().suffix(".valid-length"));
-					String validLengthString = inStream.readUTF();
-					validLength = Integer.parseInt(validLengthString);
-					System.out.println("VALID LENGTH: " + validLength);
-				}
-				FSDataInputStream inStream = dfs.open(file.getPath());
-				byte[] buffer = new byte[validLength];
-				inStream.readFully(0, buffer, 0, validLength);
-				inStream.close();
-
-				ByteArrayInputStream bais = new ByteArrayInputStream(buffer);
-
-				InputStreamReader inStreamReader = new InputStreamReader(bais);
-				BufferedReader br = new BufferedReader(inStreamReader);
-
-				String line = br.readLine();
-				while (line != null) {
-					Matcher matcher = messageRegex.matcher(line);
-					if (matcher.matches()) {
-						numRead++;
-						int messageId = Integer.parseInt(matcher.group(1));
-						readNumbers.add(messageId);
-					} else {
-						Assert.fail("Read line does not match expected pattern.");
-					}
-					line = br.readLine();
-				}
-				br.close();
-				inStreamReader.close();
-				bais.close();
-			}
-		}
-
-		// Verify that we read all strings (at-least-once)
-		Assert.assertEquals(NUM_STRINGS, readNumbers.size());
-
-		// Verify that we don't have duplicates (boom!, exactly-once)
-		Assert.assertEquals(NUM_STRINGS, numRead);
-	}
-
-	private static class OnceFailingIdentityMapper extends RichMapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		private static volatile boolean hasFailed = false;
-
-		private final long numElements;
-
-		private long failurePos;
-		private long count;
-
-
-		OnceFailingIdentityMapper(long numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void open(org.apache.flink.configuration.Configuration parameters) throws IOException {
-			long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
-			long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
-
-			failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
-			count = 0;
-		}
-
-		@Override
-		public String map(String value) throws Exception {
-			count++;
-			if (!hasFailed && count >= failurePos) {
-				hasFailed = true;
-				throw new Exception("Test Failure");
-			}
-
-			return value;
-		}
-	}
-
-	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
-			implements CheckpointedAsynchronously<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final long numElements;
-
-		private int index;
-
-		private volatile boolean isRunning = true;
-
-
-		StringGeneratingSourceFunction(long numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<String> ctx) throws Exception {
-			final Object lockingObject = ctx.getCheckpointLock();
-
-			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
-
-			if (index == 0) {
-				index = getRuntimeContext().getIndexOfThisSubtask();
-			}
-
-			while (isRunning && index < numElements) {
-
-				Thread.sleep(1);
-				synchronized (lockingObject) {
-					ctx.collect("message " + index);
-					index += step;
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		private static String randomString(StringBuilder bld, Random rnd) {
-			final int len = rnd.nextInt(10) + 5;
-
-			for (int i = 0; i < len; i++) {
-				char next = (char) (rnd.nextInt(20000) + 33);
-				bld.append(next);
-			}
-
-			return bld.toString();
-		}
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
-		}
-
-		@Override
-		public void restoreState(Integer state) {
-			index = state;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
deleted file mode 100644
index 65904d2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkFaultToleranceITCase.java
+++ /dev/null
@@ -1,284 +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.connectors.fs;
-
-import com.google.common.collect.Sets;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.test.checkpointing.StreamFaultToleranceTestBase;
-import org.apache.flink.util.NetUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.Random;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import static org.junit.Assert.assertTrue;
-
-/**
- * Tests for {@link org.apache.flink.streaming.connectors.fs.RollingSink}.
- *
- * <p>
- * This test only verifies the exactly once behaviour of the sink. Another test tests the
- * rolling behaviour.
- */
-public class RollingSinkFaultToleranceITCase extends StreamFaultToleranceTestBase {
-
-	final long NUM_STRINGS = 16_000;
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	private static MiniDFSCluster hdfsCluster;
-	private static org.apache.hadoop.fs.FileSystem dfs;
-
-	private static String outPath;
-
-
-
-	@BeforeClass
-	public static void createHDFS() throws IOException {
-		Configuration conf = new Configuration();
-
-		File dataDir = tempFolder.newFolder();
-
-		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
-		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-		hdfsCluster = builder.build();
-
-		dfs = hdfsCluster.getFileSystem();
-
-		outPath = "hdfs://"
-				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
-				+ "/string-non-rolling-out";
-	}
-
-	@AfterClass
-	public static void destroyHDFS() {
-		if (hdfsCluster != null) {
-			hdfsCluster.shutdown();
-		}
-	}
-
-
-	@Override
-	public void testProgram(StreamExecutionEnvironment env) {
-		assertTrue("Broken test setup", NUM_STRINGS % 40 == 0);
-
-		int PARALLELISM = 6;
-
-		env.enableCheckpointing(200);
-		env.setParallelism(PARALLELISM);
-		env.disableOperatorChaining();
-
-		DataStream<String> stream = env.addSource(new StringGeneratingSourceFunction(NUM_STRINGS)).startNewChain();
-
-		DataStream<String> mapped = stream
-				.map(new OnceFailingIdentityMapper(NUM_STRINGS));
-
-		RollingSink<String> sink = new RollingSink<String>(outPath)
-				.setBucketer(new NonRollingBucketer())
-				.setBatchSize(10000)
-				.setValidLengthPrefix("")
-				.setPendingPrefix("");
-
-		mapped.addSink(sink);
-
-	}
-
-	@Override
-	public void postSubmit() throws Exception {
-		// We read the files and verify that we have read all the strings. If a valid-length
-		// file exists we only read the file to that point. (This test should work with
-		// FileSystems that support truncate() and with others as well.)
-
-		Pattern messageRegex = Pattern.compile("message (\\d*)");
-
-		// Keep a set of the message IDs that we read. The size must equal the read count and
-		// the NUM_STRINGS. If numRead is bigger than the size of the set we have seen some
-		// elements twice.
-		Set<Integer> readNumbers = Sets.newHashSet();
-		int numRead = 0;
-
-		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(
-				outPath), true);
-
-		while (files.hasNext()) {
-			LocatedFileStatus file = files.next();
-
-			if (!file.getPath().toString().endsWith(".valid-length")) {
-				int validLength = (int) file.getLen();
-				if (dfs.exists(file.getPath().suffix(".valid-length"))) {
-					FSDataInputStream inStream = dfs.open(file.getPath().suffix(".valid-length"));
-					String validLengthString = inStream.readUTF();
-					validLength = Integer.parseInt(validLengthString);
-					System.out.println("VALID LENGTH: " + validLength);
-				}
-				FSDataInputStream inStream = dfs.open(file.getPath());
-				byte[] buffer = new byte[validLength];
-				inStream.readFully(0, buffer, 0, validLength);
-				inStream.close();
-
-				ByteArrayInputStream bais = new ByteArrayInputStream(buffer);
-
-				InputStreamReader inStreamReader = new InputStreamReader(bais);
-				BufferedReader br = new BufferedReader(inStreamReader);
-
-				String line = br.readLine();
-				while (line != null) {
-					Matcher matcher = messageRegex.matcher(line);
-					if (matcher.matches()) {
-						numRead++;
-						int messageId = Integer.parseInt(matcher.group(1));
-						readNumbers.add(messageId);
-					} else {
-						Assert.fail("Read line does not match expected pattern.");
-					}
-					line = br.readLine();
-				}
-				br.close();
-				inStreamReader.close();
-				bais.close();
-			}
-		}
-
-		// Verify that we read all strings (at-least-once)
-		Assert.assertEquals(NUM_STRINGS, readNumbers.size());
-
-		// Verify that we don't have duplicates (boom!, exactly-once)
-		Assert.assertEquals(NUM_STRINGS, numRead);
-	}
-
-	private static class OnceFailingIdentityMapper extends RichMapFunction<String, String> {
-		private static final long serialVersionUID = 1L;
-
-		private static volatile boolean hasFailed = false;
-
-		private final long numElements;
-
-		private long failurePos;
-		private long count;
-
-
-		OnceFailingIdentityMapper(long numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void open(org.apache.flink.configuration.Configuration parameters) throws IOException {
-			long failurePosMin = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
-			long failurePosMax = (long) (0.9 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
-
-			failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
-			count = 0;
-		}
-
-		@Override
-		public String map(String value) throws Exception {
-			count++;
-			if (!hasFailed && count >= failurePos) {
-				hasFailed = true;
-				throw new Exception("Test Failure");
-			}
-
-			return value;
-		}
-	}
-
-	private static class StringGeneratingSourceFunction extends RichParallelSourceFunction<String>
-			implements CheckpointedAsynchronously<Integer> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final long numElements;
-
-		private int index;
-
-		private volatile boolean isRunning = true;
-
-
-		StringGeneratingSourceFunction(long numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<String> ctx) throws Exception {
-			final Object lockingObject = ctx.getCheckpointLock();
-
-			final int step = getRuntimeContext().getNumberOfParallelSubtasks();
-
-			if (index == 0) {
-				index = getRuntimeContext().getIndexOfThisSubtask();
-			}
-
-			while (isRunning && index < numElements) {
-
-				Thread.sleep(1);
-				synchronized (lockingObject) {
-					ctx.collect("message " + index);
-					index += step;
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-
-		private static String randomString(StringBuilder bld, Random rnd) {
-			final int len = rnd.nextInt(10) + 5;
-
-			for (int i = 0; i < len; i++) {
-				char next = (char) (rnd.nextInt(20000) + 33);
-				bld.append(next);
-			}
-
-			return bld.toString();
-		}
-
-		@Override
-		public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
-			return index;
-		}
-
-		@Override
-		public void restoreState(Integer state) {
-			index = state;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
deleted file mode 100644
index 9770f41..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java
+++ /dev/null
@@ -1,506 +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.connectors.fs;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichFilterFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.taskmanager.MultiShotLatch;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.NetUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-
-/**
- * Tests for {@link RollingSink}. These
- * tests test the different output methods as well as the rolling feature using a manual clock
- * that increases time in lockstep with element computation using latches.
- *
- * <p>
- * This only tests the rolling behaviour of the sink. There is a separate ITCase that verifies
- * exactly once behaviour.
- */
-public class RollingSinkITCase extends StreamingMultipleProgramsTestBase {
-
-	@ClassRule
-	public static TemporaryFolder tempFolder = new TemporaryFolder();
-
-	private static MiniDFSCluster hdfsCluster;
-	private static org.apache.hadoop.fs.FileSystem dfs;
-	private static String hdfsURI;
-
-
-	@BeforeClass
-	public static void createHDFS() throws IOException {
-		Configuration conf = new Configuration();
-
-		File dataDir = tempFolder.newFolder();
-
-		conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath());
-		MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-		hdfsCluster = builder.build();
-
-		dfs = hdfsCluster.getFileSystem();
-
-		hdfsURI = "hdfs://"
-				+ NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort())
-				+ "/";
-	}
-
-	@AfterClass
-	public static void destroyHDFS() {
-		hdfsCluster.shutdown();
-	}
-
-	/**
-	 * This tests {@link StringWriter} with
-	 * non-rolling output.
-	 */
-	@Test
-	public void testNonRollingStringWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/string-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-		RollingSink<String> sink = new RollingSink<String>(outPath)
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		source
-				.map(new MapFunction<Tuple2<Integer,String>, String>() {
-					private static final long serialVersionUID = 1L;
-					@Override
-					public String map(Tuple2<Integer, String> value) throws Exception {
-						return value.f1;
-					}
-				})
-				.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-
-		BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			String line = br.readLine();
-			Assert.assertEquals("message #" + i, line);
-		}
-
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-
-		br = new BufferedReader(new InputStreamReader(inStream));
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			String line = br.readLine();
-			Assert.assertEquals("message #" + i, line);
-		}
-
-		inStream.close();
-	}
-
-	/**
-	 * This tests {@link SequenceFileWriter}
-	 * with non-rolling output and without compression.
-	 */
-	@Test
-	public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/seq-no-comp-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
-				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
-			}
-		});
-
-
-		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
-				.setWriter(new SequenceFileWriter<IntWritable, Text>())
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		mapped.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-
-		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		IntWritable intWritable = new IntWritable();
-		Text txt = new Text();
-
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-
-		reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-	}
-
-	/**
-	 * This tests {@link SequenceFileWriter}
-	 * with non-rolling output but with compression.
-	 */
-	@Test
-	public void testNonRollingSequenceFileWithCompressionWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/seq-non-rolling-out";
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new TestSourceFunction(NUM_ELEMENTS))
-				.broadcast()
-				.filter(new OddEvenFilter());
-
-		DataStream<Tuple2<IntWritable, Text>> mapped =  source.map(new MapFunction<Tuple2<Integer,String>, Tuple2<IntWritable, Text>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Tuple2<IntWritable, Text> map(Tuple2<Integer, String> value) throws Exception {
-				return Tuple2.of(new IntWritable(value.f0), new Text(value.f1));
-			}
-		});
-
-
-		RollingSink<Tuple2<IntWritable, Text>> sink = new RollingSink<Tuple2<IntWritable, Text>>(outPath)
-				.setWriter(new SequenceFileWriter<IntWritable, Text>("Default", SequenceFile.CompressionType.BLOCK))
-				.setBucketer(new NonRollingBucketer())
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		mapped.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		FSDataInputStream inStream = dfs.open(new Path(outPath + "/part-0-0"));
-
-		SequenceFile.Reader reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		IntWritable intWritable = new IntWritable();
-		Text txt = new Text();
-
-		for (int i = 0; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-
-		inStream = dfs.open(new Path(outPath + "/part-1-0"));
-
-		reader = new SequenceFile.Reader(inStream,
-				1000,
-				0,
-				100000,
-				new Configuration());
-
-		for (int i = 1; i < NUM_ELEMENTS; i += 2) {
-			reader.next(intWritable, txt);
-			Assert.assertEquals(i, intWritable.get());
-			Assert.assertEquals("message #" + i, txt.toString());
-		}
-
-		reader.close();
-		inStream.close();
-	}
-
-	// we use this to synchronize the clock changes to elements being processed
-	final static MultiShotLatch latch1 = new MultiShotLatch();
-	final static MultiShotLatch latch2 = new MultiShotLatch();
-
-	/**
-	 * This uses {@link org.apache.flink.streaming.connectors.fs.DateTimeBucketer} to
-	 * produce rolling files. The clock of DateTimeBucketer is set to
-	 * {@link ModifyableClock} to keep the time in lockstep with the processing of elements using
-	 * latches.
-	 */
-	@Test
-	public void testDateTimeRollingStringWriter() throws Exception {
-		final int NUM_ELEMENTS = 20;
-		final int PARALLELISM = 2;
-		final String outPath = hdfsURI + "/rolling-out";
-		DateTimeBucketer.setClock(new ModifyableClock());
-		ModifyableClock.setCurrentTime(0);
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(PARALLELISM);
-
-
-
-		DataStream<Tuple2<Integer, String>> source = env.addSource(new WaitingTestSourceFunction(
-				NUM_ELEMENTS))
-				.broadcast();
-
-		// the parallel flatMap is chained to the sink, so when it has seen 5 elements it can
-		// fire the latch
-		DataStream<String> mapped = source
-				.flatMap(new RichFlatMapFunction<Tuple2<Integer, String>, String>() {
-					private static final long serialVersionUID = 1L;
-
-					int count = 0;
-					@Override
-					public void flatMap(Tuple2<Integer, String> value,
-							Collector<String> out) throws Exception {
-						out.collect(value.f1);
-						count++;
-						if (count >= 5) {
-							if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
-								latch1.trigger();
-							} else {
-								latch2.trigger();
-							}
-							count = 0;
-						}
-					}
-
-				});
-
-		RollingSink<String> sink = new RollingSink<String>(outPath)
-				.setBucketer(new DateTimeBucketer("ss"))
-				.setPartPrefix("part")
-				.setPendingPrefix("")
-				.setPendingSuffix("");
-
-		mapped.addSink(sink);
-
-		env.execute("RollingSink String Write Test");
-
-		RemoteIterator<LocatedFileStatus> files = dfs.listFiles(new Path(outPath), true);
-
-		// we should have 8 rolling files, 4 time intervals and parallelism of 2
-		int numFiles = 0;
-		while (files.hasNext()) {
-			LocatedFileStatus file = files.next();
-			numFiles++;
-			if (file.getPath().toString().contains("rolling-out/00")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 0; i < 5; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/05")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 5; i < 10; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/10")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 10; i < 15; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else if (file.getPath().toString().contains("rolling-out/15")) {
-				FSDataInputStream inStream = dfs.open(file.getPath());
-
-				BufferedReader br = new BufferedReader(new InputStreamReader(inStream));
-
-				for (int i = 15; i < 20; i++) {
-					String line = br.readLine();
-					Assert.assertEquals("message #" + i, line);
-				}
-
-				inStream.close();
-			} else {
-				Assert.fail("File " + file + " does not match any expected roll pattern.");
-			}
-		}
-
-		Assert.assertEquals(8, numFiles);
-	}
-
-
-	private static class TestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean running = true;
-
-		private final int numElements;
-
-		public TestSourceFunction(int numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
-			for (int i = 0; i < numElements && running; i++) {
-				ctx.collect(Tuple2.of(i, "message #" + i));
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-
-	/**
-	 * This waits on the two multi-shot latches. The latches are triggered in a parallel
-	 * flatMap inside the test topology.
-	 */
-	private static class WaitingTestSourceFunction implements SourceFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		private volatile boolean running = true;
-
-		private final int numElements;
-
-		public WaitingTestSourceFunction(int numElements) {
-			this.numElements = numElements;
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, String>> ctx) throws Exception {
-			for (int i = 0; i < numElements && running; i++) {
-				if (i % 5 == 0 && i > 0) {
-					// update the clock after "five seconds", so we get 20 seconds in total
-					// with 5 elements in each time window
-					latch1.await();
-					latch2.await();
-					ModifyableClock.setCurrentTime(i * 1000);
-				}
-				ctx.collect(Tuple2.of(i, "message #" + i));
-			}
-		}
-
-		@Override
-		public void cancel() {
-			running = false;
-		}
-	}
-
-	public static class OddEvenFilter extends RichFilterFunction<Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Tuple2<Integer, String> value) throws Exception {
-			if (getRuntimeContext().getIndexOfThisSubtask() == 0) {
-				return value.f0 % 2 == 0;
-			} else {
-				return value.f0 % 2 == 1;
-			}
-		}
-	}
-
-	public static class ModifyableClock implements Clock {
-
-		private static volatile long currentTime = 0;
-
-		public static void setCurrentTime(long currentTime) {
-			ModifyableClock.currentTime = currentTime;
-		}
-
-		@Override
-		public long currentTimeMillis() {
-			return currentTime;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties
deleted file mode 100644
index fe60d94..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,27 +0,0 @@
-################################################################################
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-log4j.rootLogger=OFF, testlogger
-
-log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
-log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
-log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
-
-# suppress the irrelevant (wrong) warnings from the netty channel handler
-log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml
deleted file mode 100644
index 45b3b92..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +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.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.streaming" level="WARN"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/pom.xml
deleted file mode 100644
index b4f5ad2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/pom.xml
+++ /dev/null
@@ -1,174 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-flume</artifactId>
-	<name>flink-connector-flume</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<flume-ng.version>1.5.0</flume-ng.version>
-	</properties>
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flume</groupId>
-			<artifactId>flume-ng-core</artifactId>
-			<version>${flume-ng.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>commons-io</groupId>
-					<artifactId>commons-io</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>commons-codec</groupId>
-					<artifactId>commons-codec</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>commons-cli</groupId>
-					<artifactId>commons-cli</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>commons-lang</groupId>
-					<artifactId>commons-lang</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.avro</groupId>
-					<artifactId>avro</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.codehaus.jackson</groupId>
-					<artifactId>jackson-core-asl</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.codehaus.jackson</groupId>
-					<artifactId>jackson-mapper-asl</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.thoughtworks.paranamer</groupId>
-					<artifactId>paranamer</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.xerial.snappy</groupId>
-					<artifactId>snappy-java</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.tukaani</groupId>
-					<artifactId>xz</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.velocity</groupId>
-					<artifactId>velocity</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>commons-collections</groupId>
-					<artifactId>commons-collections</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>servlet-api</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-util</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.google.code.gson</groupId>
-					<artifactId>gson</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.thrift</groupId>
-					<artifactId>libthrift</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-
-			<plugin>
-				<!-- Override artifactSet configuration to build fat-jar with all dependencies packed. -->
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<configuration>
-							<artifactSet>
-								<includes combine.children="append">
-									<!-- We include all dependencies that transitively depend on guava -->
-									<include>org.apache.flume:*</include>
-								</includes>
-							</artifactSet>
-							<transformers>
-								<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
-							</transformers>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
deleted file mode 100644
index 50f5770..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSink.java
+++ /dev/null
@@ -1,141 +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.connectors.flume;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flume.Event;
-import org.apache.flume.EventDeliveryException;
-import org.apache.flume.FlumeException;
-import org.apache.flume.api.RpcClient;
-import org.apache.flume.api.RpcClientFactory;
-import org.apache.flume.event.EventBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class FlumeSink<IN> extends RichSinkFunction<IN> {
-	private static final long serialVersionUID = 1L;
-
-	private static final Logger LOG = LoggerFactory.getLogger(FlumeSink.class);
-
-	private transient FlinkRpcClientFacade client;
-	boolean initDone = false;
-	String host;
-	int port;
-	SerializationSchema<IN, byte[]> schema;
-
-	public FlumeSink(String host, int port, SerializationSchema<IN, byte[]> schema) {
-		this.host = host;
-		this.port = port;
-		this.schema = schema;
-	}
-
-	/**
-	 * Receives tuples from the Apache Flink {@link DataStream} and forwards
-	 * them to Apache Flume.
-	 * 
-	 * @param value
-	 *            The tuple arriving from the datastream
-	 */
-	@Override
-	public void invoke(IN value) {
-
-		byte[] data = schema.serialize(value);
-		client.sendDataToFlume(data);
-
-	}
-
-	private class FlinkRpcClientFacade {
-		private RpcClient client;
-		private String hostname;
-		private int port;
-
-		/**
-		 * Initializes the connection to Apache Flume.
-		 * 
-		 * @param hostname
-		 *            The host
-		 * @param port
-		 *            The port.
-		 */
-		public void init(String hostname, int port) {
-			// Setup the RPC connection
-			this.hostname = hostname;
-			this.port = port;
-			int initCounter = 0;
-			while (true) {
-				if (initCounter >= 90) {
-					throw new RuntimeException("Cannot establish connection with" + port + " at "
-							+ host);
-				}
-				try {
-					this.client = RpcClientFactory.getDefaultInstance(hostname, port);
-				} catch (FlumeException e) {
-					// Wait one second if the connection failed before the next
-					// try
-					try {
-						Thread.sleep(1000);
-					} catch (InterruptedException e1) {
-						if (LOG.isErrorEnabled()) {
-							LOG.error("Interrupted while trying to connect {} at {}", port, host);
-						}
-					}
-				}
-				if (client != null) {
-					break;
-				}
-				initCounter++;
-			}
-			initDone = true;
-		}
-
-		/**
-		 * Sends byte arrays as {@link Event} series to Apache Flume.
-		 * 
-		 * @param data
-		 *            The byte array to send to Apache FLume
-		 */
-		public void sendDataToFlume(byte[] data) {
-			Event event = EventBuilder.withBody(data);
-
-			try {
-				client.append(event);
-
-			} catch (EventDeliveryException e) {
-				// clean up and recreate the client
-				client.close();
-				client = null;
-				client = RpcClientFactory.getDefaultInstance(hostname, port);
-			}
-		}
-
-	}
-
-	@Override
-	public void close() {
-		client.client.close();
-	}
-
-	@Override
-	public void open(Configuration config) {
-		client = new FlinkRpcClientFacade();
-		client.init(host, port);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
deleted file mode 100644
index 8fecd0f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeSource.java
+++ /dev/null
@@ -1,149 +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.connectors.flume;
-//
-//import java.util.List;
-//
-//import org.apache.flink.streaming.api.datastream.DataStream;
-//import org.apache.flink.streaming.api.functions.source.ConnectorSource;
-//import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-//import org.apache.flink.util.Collector;
-//import org.apache.flume.Context;
-//import org.apache.flume.channel.ChannelProcessor;
-//import org.apache.flume.source.AvroSource;
-//import org.apache.flume.source.avro.AvroFlumeEvent;
-//import org.apache.flume.source.avro.Status;
-//
-//public class FlumeSource<OUT> extends ConnectorSource<OUT> {
-//	private static final long serialVersionUID = 1L;
-//
-//	String host;
-//	String port;
-//	volatile boolean finished = false;
-//
-//	private volatile boolean isRunning = false;
-//
-//	FlumeSource(String host, int port, DeserializationSchema<OUT> deserializationSchema) {
-//		super(deserializationSchema);
-//		this.host = host;
-//		this.port = Integer.toString(port);
-//	}
-//
-//	public class MyAvroSource extends AvroSource {
-//		Collector<OUT> output;
-//
-//		/**
-//		 * Sends the AvroFlumeEvent from it's argument list to the Apache Flink
-//		 * {@link DataStream}.
-//		 *
-//		 * @param avroEvent
-//		 *            The event that should be sent to the dataStream
-//		 * @return A {@link Status}.OK message if sending the event was
-//		 *         successful.
-//		 */
-//		@Override
-//		public Status append(AvroFlumeEvent avroEvent) {
-//			collect(avroEvent);
-//			return Status.OK;
-//		}
-//
-//		/**
-//		 * Sends the AvroFlumeEvents from it's argument list to the Apache Flink
-//		 * {@link DataStream}.
-//		 *
-//		 * @param events
-//		 *            The events that is sent to the dataStream
-//		 * @return A Status.OK message if sending the events was successful.
-//		 */
-//		@Override
-//		public Status appendBatch(List<AvroFlumeEvent> events) {
-//			for (AvroFlumeEvent avroEvent : events) {
-//				collect(avroEvent);
-//			}
-//
-//			return Status.OK;
-//		}
-//
-//		/**
-//		 * Deserializes the AvroFlumeEvent before sending it to the Apache Flink
-//		 * {@link DataStream}.
-//		 *
-//		 * @param avroEvent
-//		 *            The event that is sent to the dataStream
-//		 */
-//		private void collect(AvroFlumeEvent avroEvent) {
-//			byte[] b = avroEvent.getBody().array();
-//			OUT out = FlumeSource.this.schema.deserialize(b);
-//
-//			if (schema.isEndOfStream(out)) {
-//				FlumeSource.this.finished = true;
-//				this.stop();
-//				FlumeSource.this.notifyAll();
-//			} else {
-//				output.collect(out);
-//			}
-//
-//		}
-//
-//	}
-//
-//	MyAvroSource avroSource;
-//
-//	/**
-//	 * Configures the AvroSource. Also sets the output so the application can
-//	 * use it from outside of the invoke function.
-//	 *
-//	 * @param output
-//	 *            The output used in the invoke function
-//	 */
-//	public void configureAvroSource(Collector<OUT> output) {
-//
-//		avroSource = new MyAvroSource();
-//		avroSource.output = output;
-//		Context context = new Context();
-//		context.put("port", port);
-//		context.put("bind", host);
-//		avroSource.configure(context);
-//		// An instance of a ChannelProcessor is required for configuring the
-//		// avroSource although it will not be used in this case.
-//		ChannelProcessor cp = new ChannelProcessor(null);
-//		avroSource.setChannelProcessor(cp);
-//	}
-//
-//	/**
-//	 * Configures the AvroSource and runs until the user calls a close function.
-//	 *
-//	 * @param output
-//	 *            The Collector for sending data to the datastream
-//	 */
-//	@Override
-//	public void run(Collector<OUT> output) throws Exception {
-//		isRunning = true;
-//		configureAvroSource(output);
-//		avroSource.start();
-//		while (!finished && isRunning) {
-//			this.wait();
-//		}
-//	}
-//
-//	@Override
-//	public void cancel() {
-//		isRunning = false;
-//	}
-//
-//}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
deleted file mode 100644
index 45da6eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-flume/src/main/java/org/apache/flink/streaming/connectors/flume/FlumeTopology.java
+++ /dev/null
@@ -1,49 +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.connectors.flume;
-//
-//import org.apache.flink.streaming.api.datastream.DataStream;
-//import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-//import org.apache.flink.streaming.util.serialization.SerializationSchema;
-//import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-//
-//public class FlumeTopology {
-//
-//	public static void main(String[] args) throws Exception {
-//
-//		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
-//
-//		@SuppressWarnings("unused")
-//		DataStream<String> inputStream1 = env.addSource(
-//				new FlumeSource<String>("localhost", 41414, new SimpleStringSchema())).addSink(
-//				new FlumeSink<String>("localhost", 42424, new StringToByteSerializer()));
-//
-//		env.execute();
-//	}
-//
-//	public static class StringToByteSerializer implements SerializationSchema<String, byte[]> {
-//
-//		private static final long serialVersionUID = 1L;
-//
-//		@Override
-//		public byte[] serialize(String element) {
-//			return element.getBytes();
-//		}
-//	}
-//
-//}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml
deleted file mode 100644
index f098d9c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/pom.xml
+++ /dev/null
@@ -1,130 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-connectors-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-connector-kafka</artifactId>
-	<name>flink-connector-kafka</name>
-
-	<packaging>jar</packaging>
-
-	<!-- Allow users to pass custom connector versions -->
-	<properties>
-		<kafka.version>0.8.2.0</kafka.version>
-	</properties>
-
-	<dependencies>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.kafka</groupId>
-			<artifactId>kafka_${scala.binary.version}</artifactId>
-			<version>${kafka.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>com.sun.jmx</groupId>
-					<artifactId>jmxri</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jdmk</groupId>
-					<artifactId>jmxtools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-simple</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>net.sf.jopt-simple</groupId>
-					<artifactId>jopt-simple</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.scala-lang</groupId>
-					<artifactId>scala-reflect</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.scala-lang</groupId>
-					<artifactId>scala-compiler</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.yammer.metrics</groupId>
-					<artifactId>metrics-annotation</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.xerial.snappy</groupId>
-					<artifactId>snappy-java</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.curator</groupId>
-			<artifactId>curator-test</artifactId>
-			<version>${curator.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-failsafe-plugin</artifactId>
-				<configuration>
-					<!-- Enforce single fork execution due to heavy mini cluster use in the tests -->
-					<forkCount>1</forkCount>
-				</configuration>
-			</plugin>
-		</plugins>
-	</build>
-	
-</project>


[29/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/FieldAccessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/FieldAccessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/FieldAccessor.java
deleted file mode 100644
index ba8a55b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/FieldAccessor.java
+++ /dev/null
@@ -1,249 +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.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompositeType;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.java.typeutils.runtime.PojoComparator;
-import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
-import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase;
-
-import java.io.Serializable;
-import java.lang.reflect.Array;
-import java.util.List;
-
-import scala.Product;
-
-
-/**
- * These classes encapsulate the logic of accessing a field specified by the user as either an index
- * or a field expression string. TypeInformation can also be requested for the field.
- * The position index might specify a field of a Tuple, an array, or a simple type (only "0th field").
- */
-public abstract class FieldAccessor<R, F> implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	TypeInformation fieldType;
-
-	// Note: Returns the corresponding basic type for array of a primitive type (Integer for int[]).
-	@SuppressWarnings("unchecked")
-	public TypeInformation<F> getFieldType() {
-		return fieldType;
-	}
-
-
-	public abstract F get(R record);
-
-	// Note: This has to return the result, because the SimpleFieldAccessor might not be able to modify the
-	// record in place. (for example, when R is simply Double) (Unfortunately there is no passing by reference in Java.)
-	public abstract R set(R record, F fieldValue);
-
-
-
-	@SuppressWarnings("unchecked")
-	public static <R, F> FieldAccessor<R, F> create(int pos, TypeInformation<R> typeInfo, ExecutionConfig config) {
-		if (typeInfo.isTupleType() && ((TupleTypeInfoBase)typeInfo).isCaseClass()) {
-			return new ProductFieldAccessor<R, F>(pos, typeInfo, config);
-		} else if (typeInfo.isTupleType()) {
-			return new TupleFieldAccessor<R, F>(pos, typeInfo);
-		} else if (typeInfo instanceof BasicArrayTypeInfo || typeInfo instanceof PrimitiveArrayTypeInfo) {
-			return new ArrayFieldAccessor<R, F>(pos, typeInfo);
-		} else {
-			if(pos != 0) {
-				throw new IndexOutOfBoundsException("Not 0th field selected for a simple type (non-tuple, non-array).");
-			}
-			return (FieldAccessor<R, F>) new SimpleFieldAccessor<R>(typeInfo);
-		}
-	}
-
-	public static <R, F> FieldAccessor<R, F> create(String field, TypeInformation<R> typeInfo, ExecutionConfig config) {
-		if (typeInfo.isTupleType() && ((TupleTypeInfoBase)typeInfo).isCaseClass()) {
-			int pos = ((TupleTypeInfoBase)typeInfo).getFieldIndex(field);
-			if(pos == -2) {
-				throw new RuntimeException("Invalid field selected: " + field);
-			}
-			return new ProductFieldAccessor<R, F>(pos, typeInfo, config);
-		} else if(typeInfo.isTupleType()) {
-			return new TupleFieldAccessor<R, F>(((TupleTypeInfo)typeInfo).getFieldIndex(field), typeInfo);
-		} else {
-			return new PojoFieldAccessor<R, F>(field, typeInfo, config);
-		}
-	}
-
-
-
-	public static class SimpleFieldAccessor<R> extends FieldAccessor<R, R> {
-
-		private static final long serialVersionUID = 1L;
-
-		SimpleFieldAccessor(TypeInformation<R> typeInfo) {
-			this.fieldType = typeInfo;
-		}
-
-		@Override
-		public R get(R record) {
-			return record;
-		}
-
-		@Override
-		public R set(R record, R fieldValue) {
-			return fieldValue;
-		}
-	}
-
-	public static class ArrayFieldAccessor<R, F> extends FieldAccessor<R, F> {
-
-		private static final long serialVersionUID = 1L;
-
-		int pos;
-
-		ArrayFieldAccessor(int pos, TypeInformation typeInfo) {
-			this.pos = pos;
-			this.fieldType = BasicTypeInfo.getInfoFor(typeInfo.getTypeClass().getComponentType());
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public F get(R record) {
-			return (F) Array.get(record, pos);
-		}
-
-		@Override
-		public R set(R record, F fieldValue) {
-			Array.set(record, pos, fieldValue);
-			return record;
-		}
-	}
-
-	public static class TupleFieldAccessor<R, F> extends FieldAccessor<R, F> {
-
-		private static final long serialVersionUID = 1L;
-
-		int pos;
-
-		TupleFieldAccessor(int pos, TypeInformation<R> typeInfo) {
-			this.pos = pos;
-			this.fieldType = ((TupleTypeInfo)typeInfo).getTypeAt(pos);
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public F get(R record) {
-			Tuple tuple = (Tuple) record;
-			return (F)tuple.getField(pos);
-		}
-
-		@Override
-		public R set(R record, F fieldValue) {
-			Tuple tuple = (Tuple) record;
-			tuple.setField(fieldValue, pos);
-			return record;
-		}
-	}
-
-	public static class PojoFieldAccessor<R, F> extends FieldAccessor<R, F> {
-
-		private static final long serialVersionUID = 1L;
-
-		PojoComparator comparator;
-
-		PojoFieldAccessor(String field, TypeInformation<R> type, ExecutionConfig config) {
-			if (!(type instanceof CompositeType<?>)) {
-				throw new IllegalArgumentException(
-						"Key expressions are only supported on POJO types and Tuples. "
-								+ "A type is considered a POJO if all its fields are public, or have both getters and setters defined");
-			}
-
-			@SuppressWarnings("unchecked")
-			CompositeType<R> cType = (CompositeType<R>) type;
-
-			List<CompositeType.FlatFieldDescriptor> fieldDescriptors = cType.getFlatFields(field);
-
-			int logicalKeyPosition = fieldDescriptors.get(0).getPosition();
-			this.fieldType = fieldDescriptors.get(0).getType();
-			Class<?> keyClass = fieldType.getTypeClass();
-
-			if (cType instanceof PojoTypeInfo) {
-				comparator = (PojoComparator<R>) cType.createComparator(
-						new int[] { logicalKeyPosition }, new boolean[] { false }, 0, config);
-			} else {
-				throw new IllegalArgumentException(
-						"Key expressions are only supported on POJO types. "
-								+ "A type is considered a POJO if all its fields are public, or have both getters and setters defined");
-			}
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public F get(R record) {
-			return (F) comparator.accessField(comparator.getKeyFields()[0], record);
-		}
-
-		@Override
-		public R set(R record, F fieldValue) {
-			try {
-				comparator.getKeyFields()[0].set(record, fieldValue);
-			} catch (IllegalAccessException e) {
-				throw new RuntimeException("Could not modify the specified field.", e);
-			}
-			return record;
-		}
-	}
-
-	public static class ProductFieldAccessor<R, F> extends FieldAccessor<R, F> {
-
-		private static final long serialVersionUID = 1L;
-
-		int pos;
-		TupleSerializerBase<R> serializer;
-		Object[] fields;
-		int length;
-
-		ProductFieldAccessor(int pos, TypeInformation<R> typeInfo, ExecutionConfig config) {
-			this.pos = pos;
-			this.fieldType = ((TupleTypeInfoBase<R>)typeInfo).getTypeAt(pos);
-			this.serializer = (TupleSerializerBase<R>)typeInfo.createSerializer(config);
-			this.length = this.serializer.getArity();
-			this.fields = new Object[this.length];
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public F get(R record) {
-			return (F)((Product)record).productElement(pos);
-		}
-
-		@Override
-		public R set(R record, F fieldValue) {
-			Product prod = (Product)record;
-			for (int i = 0; i < length; i++) {
-				fields[i] = prod.productElement(i);
-			}
-			fields[pos] = fieldValue;
-			return serializer.createInstance(fields);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java
deleted file mode 100644
index afbd8ab..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java
+++ /dev/null
@@ -1,240 +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.keys;
-
-import java.lang.reflect.Array;
-import java.util.Arrays;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompositeType;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * Utility class that contains helper methods to manipulating {@link KeySelector} for streaming.
- */
-public final class KeySelectorUtil {
-
-	public static <X> KeySelector<X, Tuple> getSelectorForKeys(Keys<X> keys, TypeInformation<X> typeInfo, ExecutionConfig executionConfig) {
-		if (!(typeInfo instanceof CompositeType)) {
-			throw new InvalidTypesException(
-					"This key operation requires a composite type such as Tuples, POJOs, or Case Classes.");
-		}
-
-		CompositeType<X> compositeType = (CompositeType<X>) typeInfo;
-		
-		int[] logicalKeyPositions = keys.computeLogicalKeyPositions();
-		int numKeyFields = logicalKeyPositions.length;
-		
-		// use ascending order here, the code paths for that are usually a slight bit faster
-		boolean[] orders = new boolean[numKeyFields];
-		TypeInformation<?>[] typeInfos = new TypeInformation<?>[numKeyFields];
-		for (int i = 0; i < numKeyFields; i++) {
-			orders[i] = true;
-			typeInfos[i] = compositeType.getTypeAt(logicalKeyPositions[i]);
-		}
-
-		TypeComparator<X> comparator = compositeType.createComparator(logicalKeyPositions, orders, 0, executionConfig);
-		return new ComparableKeySelector<>(comparator, numKeyFields, new TupleTypeInfo<>(typeInfos));
-	}
-
-	public static <X> ArrayKeySelector<X> getSelectorForArray(int[] positions, TypeInformation<X> typeInfo) {
-		if (positions == null || positions.length == 0 || positions.length > Tuple.MAX_ARITY) {
-			throw new IllegalArgumentException("Array keys must have between 1 and " + Tuple.MAX_ARITY + " fields.");
-		}
-		
-		TypeInformation<?> componentType;
-		
-		if (typeInfo instanceof BasicArrayTypeInfo) {
-			BasicArrayTypeInfo<X, ?>  arrayInfo = (BasicArrayTypeInfo<X, ?>) typeInfo;
-			componentType = arrayInfo.getComponentInfo();
-		}
-		else if (typeInfo instanceof PrimitiveArrayTypeInfo) {
-			PrimitiveArrayTypeInfo<X> arrayType = (PrimitiveArrayTypeInfo<X>) typeInfo;
-			componentType = arrayType.getComponentType();
-		}
-		else {
-			throw new IllegalArgumentException("This method only supports arrays of primitives and boxed primitives.");
-		}
-		
-		TypeInformation<?>[] primitiveInfos = new TypeInformation<?>[positions.length];
-		Arrays.fill(primitiveInfos, componentType);
-
-		return new ArrayKeySelector<>(positions, new TupleTypeInfo<>(primitiveInfos));
-	}
-
-	
-	public static <X, K> KeySelector<X, K> getSelectorForOneKey(
-			Keys<X> keys, Partitioner<K> partitioner, TypeInformation<X> typeInfo, ExecutionConfig executionConfig)
-	{
-		if (!(typeInfo instanceof CompositeType)) {
-			throw new InvalidTypesException(
-					"This key operation requires a composite type such as Tuples, POJOs, case classes, etc");
-		}
-		if (partitioner != null) {
-			keys.validateCustomPartitioner(partitioner, null);
-		}
-
-		CompositeType<X> compositeType = (CompositeType<X>) typeInfo;
-		int[] logicalKeyPositions = keys.computeLogicalKeyPositions();
-		if (logicalKeyPositions.length != 1) {
-			throw new IllegalArgumentException("There must be exactly 1 key specified");
-		}
-		
-		TypeComparator<X> comparator = compositeType.createComparator(
-				logicalKeyPositions, new boolean[] { true }, 0, executionConfig);
-		return new OneKeySelector<>(comparator);
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Private constructor to prevent instantiation.
-	 */
-	private KeySelectorUtil() {
-		throw new RuntimeException();
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Key extractor that extracts a single field via a generic comparator. 
-	 * 
-	 * @param <IN> The type of the elements where the key is extracted from.
-	 * @param <K> The type of the key.
-	 */
-	public static final class OneKeySelector<IN, K> implements KeySelector<IN, K> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final TypeComparator<IN> comparator;
-
-		/** Reusable array to hold the key objects. Since this is initially empty (all positions
-		 * are null), it does not have any serialization problems */
-		@SuppressWarnings("NonSerializableFieldInSerializableClass")
-		private final Object[] keyArray;
-		
-		OneKeySelector(TypeComparator<IN> comparator) {
-			this.comparator = comparator;
-			this.keyArray = new Object[1];
-		}
-
-		@Override
-		@SuppressWarnings("unchecked")
-		public K getKey(IN value) throws Exception {
-			comparator.extractKeys(value, keyArray, 0);
-			return (K) keyArray[0];
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * A key selector for selecting key fields via a TypeComparator.
-	 *
-	 * @param <IN> The type from which the key is extracted.
-	 */
-	public static final class ComparableKeySelector<IN> implements KeySelector<IN, Tuple>, ResultTypeQueryable<Tuple> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final TypeComparator<IN> comparator;
-		private final int keyLength;
-		private transient TupleTypeInfo<Tuple> tupleTypeInfo;
-
-		/** Reusable array to hold the key objects. Since this is initially empty (all positions
-		 * are null), it does not have any serialization problems */
-		@SuppressWarnings("NonSerializableFieldInSerializableClass")
-		private final Object[] keyArray;
-
-		ComparableKeySelector(TypeComparator<IN> comparator, int keyLength, TupleTypeInfo<Tuple> tupleTypeInfo) {
-			this.comparator = comparator;
-			this.keyLength = keyLength;
-			this.tupleTypeInfo = tupleTypeInfo;
-			this.keyArray = new Object[keyLength];
-		}
-
-		@Override
-		public Tuple getKey(IN value) throws Exception {
-			Tuple key = Tuple.getTupleClass(keyLength).newInstance();
-			comparator.extractKeys(value, keyArray, 0);
-			for (int i = 0; i < keyLength; i++) {
-				key.setField(keyArray[i], i);
-			}
-			return key;
-		}
-
-		@Override
-		public TypeInformation<Tuple> getProducedType() {
-			if (tupleTypeInfo == null) {
-				throw new IllegalStateException("The return type information is not available after serialization");
-			}
-			return tupleTypeInfo;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * A key selector for selecting individual array fields as keys and returns them as a Tuple.
-	 * 
-	 * @param <IN> The type from which the key is extracted, i.e., the array type.
-	 */
-	public static final class ArrayKeySelector<IN> implements KeySelector<IN, Tuple>, ResultTypeQueryable<Tuple> {
-
-		private static final long serialVersionUID = 1L;
-		
-		private final int[] fields;
-		private final Class<? extends Tuple> tupleClass;
-		private transient TupleTypeInfo<Tuple> returnType;
-
-		ArrayKeySelector(int[] fields, TupleTypeInfo<Tuple> returnType) {
-			this.fields = requireNonNull(fields);
-			this.returnType = requireNonNull(returnType);
-			this.tupleClass = Tuple.getTupleClass(fields.length);
-		}
-
-		@Override
-		public Tuple getKey(IN value) throws Exception {
-			Tuple key = tupleClass.newInstance();
-			for (int i = 0; i < fields.length; i++) {
-				key.setField(Array.get(value, fields[i]), i);
-			}
-			return key;
-		}
-
-		@Override
-		public TypeInformation<Tuple> getProducedType() {
-			if (returnType == null) {
-				throw new IllegalStateException("The return type information is not available after serialization");
-			}
-			return returnType;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java
deleted file mode 100644
index f0e4477..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java
+++ /dev/null
@@ -1,49 +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.serialization;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-
-/**
- * The deserialization schema describes how to turn the byte messages delivered by certain
- * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are
- * processed by Flink.
- * 
- * @param <T> The type created by the deserialization schema.
- */
-public interface DeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
-
-	/**
-	 * Deserializes the byte message.
-	 * 
-	 * @param message The message, as a byte array.
-	 * @return The deserialized message as an object.
-	 */
-	T deserialize(byte[] message);
-
-	/**
-	 * Method to decide whether the element signals the end of the stream. If
-	 * true is returned the element won't be emitted.
-	 * 
-	 * @param nextElement The element to test for the end-of-stream signal.
-	 * @return True, if the element signals end of stream, false otherwise.
-	 */
-	boolean isEndOfStream(T nextElement);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/JavaDefaultStringSchema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/JavaDefaultStringSchema.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/JavaDefaultStringSchema.java
deleted file mode 100644
index ebb785c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/JavaDefaultStringSchema.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.commons.lang3.SerializationUtils;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-public class JavaDefaultStringSchema implements DeserializationSchema<String>, SerializationSchema<String, byte[]> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public boolean isEndOfStream(String nextElement) {
-		return nextElement.equals("q");
-	}
-
-	@Override
-	public byte[] serialize(String element) {
-		return SerializationUtils.serialize(element);
-	}
-
-	@Override
-	public String deserialize(byte[] message) {
-		return SerializationUtils.deserialize(message);
-	}
-
-	@Override
-	public TypeInformation<String> getProducedType() {
-		return BasicTypeInfo.STRING_TYPE_INFO;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/RawSchema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/RawSchema.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/RawSchema.java
deleted file mode 100644
index 4d9aaee..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/RawSchema.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-/**
- * A "no-op" serialization and deserialization schema for byte strings. The serialized representation is
- * identical with the original representation.
- * 
- * <p>This schema never considers a byte string to signal end-of-stream.</p>
- */
-public class RawSchema implements DeserializationSchema<byte[]>, SerializationSchema<byte[], byte[]> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public byte[] deserialize(byte[] message) {
-		return message;
-	}
-
-	@Override
-	public boolean isEndOfStream(byte[] nextElement) {
-		return false;
-	}
-
-	@Override
-	public byte[] serialize(byte[] element) {
-		return element;
-	}
-
-	@Override
-	public TypeInformation<byte[]> getProducedType() {
-		return PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java
deleted file mode 100644
index 21342b2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import java.io.Serializable;
-
-/**
- * The serialization schema describes how to turn a data object into a different serialized
- * representation. Most data sinks (for example Apache Kafka) require the data to be handed
- * to them in a specific format (for example as byte strings).
- * 
- * @param <T> The type to be serialized.
- * @param <R> The serialized representation type.
- */
-public interface SerializationSchema<T, R> extends Serializable {
-
-	/**
-	 * Serializes the incoming element to a specified type.
-	 * 
-	 * @param element
-	 *            The incoming element to be serialized
-	 * @return The serialized element.
-	 */
-	R serialize(T element);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java
deleted file mode 100644
index 51d2d7f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-public class SimpleStringSchema implements DeserializationSchema<String>,
-		SerializationSchema<String, String> {
-
-	private static final long serialVersionUID = 1L;
-
-	@Override
-	public String deserialize(byte[] message) {
-		return new String(message);
-	}
-
-	@Override
-	public boolean isEndOfStream(String nextElement) {
-		return false;
-	}
-
-	@Override
-	public String serialize(String element) {
-		return element;
-	}
-
-	@Override
-	public TypeInformation<String> getProducedType() {
-		return BasicTypeInfo.STRING_TYPE_INFO;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
deleted file mode 100644
index 6ff9712..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-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.typeutils.runtime.ByteArrayInputView;
-import org.apache.flink.runtime.util.DataOutputSerializer;
-
-import java.io.IOException;
-
-/**
- * A serialization and deserialization schema that uses Flink's serialization stack to
- * transform typed from and to byte arrays.
- * 
- * @param <T> The type to be serialized.
- */
-public class TypeInformationSerializationSchema<T> implements DeserializationSchema<T>, SerializationSchema<T, byte[]> {
-	
-	private static final long serialVersionUID = -5359448468131559102L;
-	
-	/** The serializer for the actual de-/serialization */
-	private final TypeSerializer<T> serializer;
-
-	/** The reusable output serialization buffer */
-	private transient DataOutputSerializer dos;
-
-	/** The type information, to be returned by {@link #getProducedType()}. It is
-	 * transient, because it is not serializable. Note that this means that the type information
-	 * is not available at runtime, but only prior to the first serialization / deserialization */
-	private transient TypeInformation<T> typeInfo;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new de-/serialization schema for the given type.
-	 * 
-	 * @param typeInfo The type information for the type de-/serialized by this schema.
-	 * @param ec The execution config, which is used to parametrize the type serializers.
-	 */
-	public TypeInformationSerializationSchema(TypeInformation<T> typeInfo, ExecutionConfig ec) {
-		this.typeInfo = typeInfo;
-		this.serializer = typeInfo.createSerializer(ec);
-	}
-
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public T deserialize(byte[] message) {
-		try {
-			return serializer.deserialize(new ByteArrayInputView(message));
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Unable to deserialize message", e);
-		}
-	}
-
-	/**
-	 * This schema never considers an element to signal end-of-stream, so this method returns always false.
-	 * @param nextElement The element to test for the end-of-stream signal.
-	 * @return Returns false.
-	 */
-	@Override
-	public boolean isEndOfStream(T nextElement) {
-		return false;
-	}
-
-	@Override
-	public byte[] serialize(T element) {
-		if (dos == null) {
-			dos = new DataOutputSerializer(16);
-		}
-		
-		try {
-			serializer.serialize(element, dos);
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Unable to serialize record", e);
-		}
-		
-		byte[] ret = dos.getByteArray();
-		if (ret.length != dos.length()) {
-			byte[] n = new byte[dos.length()];
-			System.arraycopy(ret, 0, n, 0, dos.length());
-			ret = n;
-		}
-		dos.clear();
-		return ret;
-	}
-
-	@Override
-	public TypeInformation<T> getProducedType() {
-		if (typeInfo != null) {
-			return typeInfo;
-		}
-		else {
-			throw new IllegalStateException(
-					"The type information is not available after this class has been serialized and distributed.");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
deleted file mode 100644
index 1187fe6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
+++ /dev/null
@@ -1,228 +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.
-*/
-
-// We have it in this package because we could not mock the methods otherwise
-package org.apache.flink.runtime.io.network.partition.consumer;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.runtime.event.AbstractEvent;
-import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
-import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
-import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer;
-import org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer;
-import org.apache.flink.runtime.io.network.buffer.Buffer;
-import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
-import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.io.IOException;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Test {@link InputGate} that allows setting multiple channels. Use
- * {@link #sendElement(Object, int)} to offer an element on a specific channel. Use
- * {@link #sendEvent(AbstractEvent, int)} to offer an event on the specified channel. Use
- * {@link #endInput()} to notify all channels of input end.
- */
-public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
-
-	private final int numInputChannels;
-
-	private final TestInputChannel[] inputChannels;
-
-	private final int bufferSize;
-
-	private TypeSerializer<T> serializer;
-
-	private ConcurrentLinkedQueue<InputValue<Object>>[] inputQueues;
-
-	@SuppressWarnings("unchecked")
-	public StreamTestSingleInputGate(
-			int numInputChannels,
-			int bufferSize,
-			TypeSerializer<T> serializer) throws IOException, InterruptedException {
-		super(numInputChannels, false);
-
-		this.bufferSize = bufferSize;
-		this.serializer = serializer;
-
-		this.numInputChannels = numInputChannels;
-		inputChannels = new TestInputChannel[numInputChannels];
-
-		inputQueues = new ConcurrentLinkedQueue[numInputChannels];
-
-		setupInputChannels();
-		doReturn(bufferSize).when(inputGate).getPageSize();
-	}
-
-	@SuppressWarnings("unchecked")
-	private void setupInputChannels() throws IOException, InterruptedException {
-
-		for (int i = 0; i < numInputChannels; i++) {
-			final int channelIndex = i;
-			final RecordSerializer<SerializationDelegate<Object>> recordSerializer = new SpanningRecordSerializer<SerializationDelegate<Object>>();
-			final SerializationDelegate<Object> delegate = (SerializationDelegate<Object>) (SerializationDelegate<?>)
-					new SerializationDelegate<StreamElement>(new MultiplexingStreamRecordSerializer<T>(serializer));
-
-			inputQueues[channelIndex] = new ConcurrentLinkedQueue<InputValue<Object>>();
-			inputChannels[channelIndex] = new TestInputChannel(inputGate, i);
-
-
-			final Answer<Buffer> answer = new Answer<Buffer>() {
-				@Override
-				public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
-					InputValue<Object> input = inputQueues[channelIndex].poll();
-					if (input != null && input.isStreamEnd()) {
-						when(inputChannels[channelIndex].getInputChannel().isReleased()).thenReturn(
-								true);
-						return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
-					}
-					else if (input != null && input.isStreamRecord()) {
-						Object inputElement = input.getStreamRecord();
-						final Buffer buffer = new Buffer(
-								MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
-								mock(BufferRecycler.class));
-						
-						recordSerializer.setNextBuffer(buffer);
-						delegate.setInstance(inputElement);
-						recordSerializer.addRecord(delegate);
-
-						// Call getCurrentBuffer to ensure size is set
-						return recordSerializer.getCurrentBuffer();
-					}
-					else if (input != null && input.isEvent()) {
-						AbstractEvent event = input.getEvent();
-						return EventSerializer.toBuffer(event);
-					}
-					else {
-						synchronized (inputQueues[channelIndex]) {
-							inputQueues[channelIndex].wait();
-							return answer(invocationOnMock);
-						}
-					}
-				}
-			};
-
-			when(inputChannels[channelIndex].getInputChannel().getNextBuffer()).thenAnswer(answer);
-
-			inputGate.setInputChannel(new IntermediateResultPartitionID(),
-					inputChannels[channelIndex].getInputChannel());
-		}
-	}
-
-	public void sendElement(Object element, int channel) {
-		synchronized (inputQueues[channel]) {
-			inputQueues[channel].add(InputValue.element(element));
-			inputQueues[channel].notifyAll();
-		}
-		inputGate.onAvailableBuffer(inputChannels[channel].getInputChannel());
-	}
-
-	public void sendEvent(AbstractEvent event, int channel) {
-		synchronized (inputQueues[channel]) {
-			inputQueues[channel].add(InputValue.event(event));
-			inputQueues[channel].notifyAll();
-		}
-		inputGate.onAvailableBuffer(inputChannels[channel].getInputChannel());
-	}
-
-	public void endInput() {
-		for (int i = 0; i < numInputChannels; i++) {
-			synchronized (inputQueues[i]) {
-				inputQueues[i].add(InputValue.streamEnd());
-				inputQueues[i].notifyAll();
-			}
-			inputGate.onAvailableBuffer(inputChannels[i].getInputChannel());
-		}
-	}
-
-	/**
-	 * Returns true iff all input queues are empty.
-	 */
-	public boolean allQueuesEmpty() {
-//		for (int i = 0; i < numInputChannels; i++) {
-//			synchronized (inputQueues[i]) {
-//				inputQueues[i].add(InputValue.<T>event(new DummyEvent()));
-//				inputQueues[i].notifyAll();
-//				inputGate.onAvailableBuffer(inputChannels[i].getInputChannel());
-//			}
-//		}
-
-		for (int i = 0; i < numInputChannels; i++) {
-			if (inputQueues[i].size() > 0) {
-				return false;
-			}
-		}
-		return true;
-	}
-
-	public static class InputValue<T> {
-		private Object elementOrEvent;
-		private boolean isStreamEnd;
-		private boolean isStreamRecord;
-		private boolean isEvent;
-
-		private InputValue(Object elementOrEvent, boolean isStreamEnd, boolean isEvent, boolean isStreamRecord) {
-			this.elementOrEvent = elementOrEvent;
-			this.isStreamEnd = isStreamEnd;
-			this.isStreamRecord = isStreamRecord;
-			this.isEvent = isEvent;
-		}
-
-		public static <X> InputValue<X> element(Object element) {
-			return new InputValue<X>(element, false, false, true);
-		}
-
-		public static <X> InputValue<X> streamEnd() {
-			return new InputValue<X>(null, true, false, false);
-		}
-
-		public static <X> InputValue<X> event(AbstractEvent event) {
-			return new InputValue<X>(event, false, true, false);
-		}
-
-		public Object getStreamRecord() {
-			return elementOrEvent;
-		}
-
-		public AbstractEvent getEvent() {
-			return (AbstractEvent) elementOrEvent;
-		}
-
-		public boolean isStreamEnd() {
-			return isStreamEnd;
-		}
-
-		public boolean isStreamRecord() {
-			return isStreamRecord;
-		}
-
-		public boolean isEvent() {
-			return isEvent;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
deleted file mode 100644
index dd8dec9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
+++ /dev/null
@@ -1,416 +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;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.google.common.collect.ImmutableList;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType;
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
-import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
-import org.apache.flink.streaming.util.MockContext;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-
-import org.junit.Test;
-
-public class AggregationFunctionTest {
-
-	@Test
-	public void groupSumIntegerTest() {
-
-		// preparing expected outputs
-		List<Tuple2<Integer, Integer>> expectedGroupSumList = new ArrayList<>();
-		List<Tuple2<Integer, Integer>> expectedGroupMinList = new ArrayList<>();
-		List<Tuple2<Integer, Integer>> expectedGroupMaxList = new ArrayList<>();
-
-		int groupedSum0 = 0;
-		int groupedSum1 = 0;
-		int groupedSum2 = 0;
-
-		for (int i = 0; i < 9; i++) {
-			int groupedSum;
-			switch (i % 3) {
-				case 0:
-					groupedSum = groupedSum0 += i;
-					break;
-				case 1:
-					groupedSum = groupedSum1 += i;
-					break;
-				default:
-					groupedSum = groupedSum2 += i;
-					break;
-			}
-
-			expectedGroupSumList.add(new Tuple2<>(i % 3, groupedSum));
-			expectedGroupMinList.add(new Tuple2<>(i % 3, i % 3));
-			expectedGroupMaxList.add(new Tuple2<>(i % 3, i));
-		}
-
-		// some necessary boiler plate
-		TypeInformation<Tuple2<Integer, Integer>> typeInfo = TypeExtractor.getForObject(new Tuple2<>(0, 0));
-
-		ExecutionConfig config = new ExecutionConfig();
-
-		KeySelector<Tuple2<Integer, Integer>, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
-				new Keys.ExpressionKeys<>(new int[]{0}, typeInfo),
-				typeInfo, config);
-		TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
-
-		// aggregations tested
-		ReduceFunction<Tuple2<Integer, Integer>> sumFunction =
-				new SumAggregator<>(1, typeInfo, config);
-		ReduceFunction<Tuple2<Integer, Integer>> minFunction = new ComparableAggregator<>(
-				1, typeInfo, AggregationType.MIN, config);
-		ReduceFunction<Tuple2<Integer, Integer>> maxFunction = new ComparableAggregator<>(
-				1, typeInfo, AggregationType.MAX, config);
-
-		List<Tuple2<Integer, Integer>> groupedSumList = MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(sumFunction, typeInfo.createSerializer(config)),
-				getInputList(),
-				keySelector, keyType);
-
-		List<Tuple2<Integer, Integer>> groupedMinList = MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(minFunction, typeInfo.createSerializer(config)),
-				getInputList(),
-				keySelector, keyType);
-
-		List<Tuple2<Integer, Integer>> groupedMaxList = MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(maxFunction, typeInfo.createSerializer(config)),
-				getInputList(),
-				keySelector, keyType);
-
-		assertEquals(expectedGroupSumList, groupedSumList);
-		assertEquals(expectedGroupMinList, groupedMinList);
-		assertEquals(expectedGroupMaxList, groupedMaxList);
-	}
-
-	@Test
-	public void pojoGroupSumIntegerTest() {
-
-		// preparing expected outputs
-		List<MyPojo> expectedGroupSumList = new ArrayList<>();
-		List<MyPojo> expectedGroupMinList = new ArrayList<>();
-		List<MyPojo> expectedGroupMaxList = new ArrayList<>();
-
-		int groupedSum0 = 0;
-		int groupedSum1 = 0;
-		int groupedSum2 = 0;
-
-		for (int i = 0; i < 9; i++) {
-			int groupedSum;
-			switch (i % 3) {
-				case 0:
-					groupedSum = groupedSum0 += i;
-					break;
-				case 1:
-					groupedSum = groupedSum1 += i;
-					break;
-				default:
-					groupedSum = groupedSum2 += i;
-					break;
-			}
-
-			expectedGroupSumList.add(new MyPojo(i % 3, groupedSum));
-			expectedGroupMinList.add(new MyPojo(i % 3, i % 3));
-			expectedGroupMaxList.add(new MyPojo(i % 3, i));
-		}
-
-		// some necessary boiler plate
-		TypeInformation<MyPojo> typeInfo = TypeExtractor.getForObject(new MyPojo(0, 0));
-
-		ExecutionConfig config = new ExecutionConfig();
-
-		KeySelector<MyPojo, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
-				new Keys.ExpressionKeys<>(new String[]{"f0"}, typeInfo),
-				typeInfo, config);
-		TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
-
-		// aggregations tested
-		ReduceFunction<MyPojo> sumFunction = new SumAggregator<>("f1", typeInfo, config);
-		ReduceFunction<MyPojo> minFunction = new ComparableAggregator<>("f1", typeInfo, AggregationType.MIN,
-				false, config);
-		ReduceFunction<MyPojo> maxFunction = new ComparableAggregator<>("f1", typeInfo, AggregationType.MAX,
-				false, config);
-
-		List<MyPojo> groupedSumList = MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(sumFunction, typeInfo.createSerializer(config)),
-				getInputPojoList(),
-				keySelector, keyType);
-		
-		List<MyPojo> groupedMinList = MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(minFunction, typeInfo.createSerializer(config)),
-				getInputPojoList(),
-				keySelector, keyType);
-
-		List<MyPojo> groupedMaxList = MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(maxFunction, typeInfo.createSerializer(config)),
-				getInputPojoList(),
-				keySelector, keyType);
-
-		assertEquals(expectedGroupSumList, groupedSumList);
-		assertEquals(expectedGroupMinList, groupedMinList);
-		assertEquals(expectedGroupMaxList, groupedMaxList);
-	}
-	
-	@Test
-	public void minMaxByTest() {
-		// Tuples are grouped on field 0, aggregated on field 1
-		
-		// preparing expected outputs
-		List<Tuple3<Integer, Integer, Integer>> maxByFirstExpected = ImmutableList.of(
-				Tuple3.of(0,0,0), Tuple3.of(0,1,1), Tuple3.of(0,2,2),
-				Tuple3.of(0,2,2), Tuple3.of(0,2,2), Tuple3.of(0,2,2),
-				Tuple3.of(0,2,2), Tuple3.of(0,2,2), Tuple3.of(0,2,2));
-
-		List<Tuple3<Integer, Integer, Integer>> maxByLastExpected = ImmutableList.of(
-				Tuple3.of(0, 0, 0), Tuple3.of(0, 1, 1), Tuple3.of(0, 2, 2),
-				Tuple3.of(0, 2, 2), Tuple3.of(0, 2, 2), Tuple3.of(0, 2, 5),
-				Tuple3.of(0, 2, 5), Tuple3.of(0, 2, 5), Tuple3.of(0, 2, 8));
-
-		List<Tuple3<Integer, Integer, Integer>> minByFirstExpected = ImmutableList.of(
-				Tuple3.of(0,0,0), Tuple3.of(0,0,0), Tuple3.of(0,0,0),
-				Tuple3.of(0,0,0), Tuple3.of(0,0,0), Tuple3.of(0,0,0),
-				Tuple3.of(0,0,0), Tuple3.of(0,0,0), Tuple3.of(0,0,0));
-
-		List<Tuple3<Integer, Integer, Integer>> minByLastExpected = ImmutableList.of(
-				Tuple3.of(0, 0, 0), Tuple3.of(0, 0, 0), Tuple3.of(0, 0, 0),
-				Tuple3.of(0, 0, 3), Tuple3.of(0, 0, 3), Tuple3.of(0, 0, 3),
-				Tuple3.of(0, 0, 6), Tuple3.of(0, 0, 6), Tuple3.of(0, 0, 6));
-
-		// some necessary boiler plate
-		TypeInformation<Tuple3<Integer, Integer, Integer>> typeInfo = TypeExtractor
-				.getForObject(Tuple3.of(0,0,0));
-
-		ExecutionConfig config = new ExecutionConfig();
-
-		KeySelector<Tuple3<Integer, Integer, Integer>, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
-				new Keys.ExpressionKeys<>(new int[]{0}, typeInfo),
-				typeInfo, config);
-		TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
-		
-		// aggregations tested
-		ReduceFunction<Tuple3<Integer, Integer, Integer>> maxByFunctionFirst =
-				new ComparableAggregator<>(1, typeInfo, AggregationType.MAXBY, true, config);
-		ReduceFunction<Tuple3<Integer, Integer, Integer>> maxByFunctionLast =
-				new ComparableAggregator<>(1, typeInfo, AggregationType.MAXBY, false, config);
-		ReduceFunction<Tuple3<Integer, Integer, Integer>> minByFunctionFirst =
-				new ComparableAggregator<>(1, typeInfo, AggregationType.MINBY, true, config);
-		ReduceFunction<Tuple3<Integer, Integer, Integer>> minByFunctionLast =
-				new ComparableAggregator<>(1, typeInfo, AggregationType.MINBY, false, config);
-
-		assertEquals(maxByFirstExpected, MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(maxByFunctionFirst, typeInfo.createSerializer(config)),
-				getInputByList(),
-				keySelector, keyType));
-		
-		assertEquals(maxByLastExpected, MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(maxByFunctionLast, typeInfo.createSerializer(config)),
-				getInputByList(),
-				keySelector, keyType));
-		
-		assertEquals(minByLastExpected, MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(minByFunctionLast, typeInfo.createSerializer(config)),
-				getInputByList(),
-				keySelector, keyType));
-		
-		assertEquals(minByFirstExpected, MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(minByFunctionFirst, typeInfo.createSerializer(config)),
-				getInputByList(),
-				keySelector, keyType));
-	}
-
-	@Test
-	public void pojoMinMaxByTest() {
-		// Pojos are grouped on field 0, aggregated on field 1
-
-		// preparing expected outputs
-		List<MyPojo3> maxByFirstExpected = ImmutableList.of(
-				new MyPojo3(0, 0), new MyPojo3(1, 1), new MyPojo3(2, 2),
-				new MyPojo3(2, 2), new MyPojo3(2, 2), new MyPojo3(2, 2),
-				new MyPojo3(2, 2), new MyPojo3(2, 2), new MyPojo3(2, 2));
-
-		List<MyPojo3> maxByLastExpected = ImmutableList.of(
-				new MyPojo3(0, 0), new MyPojo3(1, 1), new MyPojo3(2, 2),
-				new MyPojo3(2, 2), new MyPojo3(2, 2), new MyPojo3(2, 5),
-				new MyPojo3(2, 5), new MyPojo3(2, 5), new MyPojo3(2, 8));
-
-		List<MyPojo3> minByFirstExpected = ImmutableList.of(
-				new MyPojo3(0, 0), new MyPojo3(0, 0), new MyPojo3(0, 0),
-				new MyPojo3(0, 0), new MyPojo3(0, 0), new MyPojo3(0, 0),
-				new MyPojo3(0, 0), new MyPojo3(0, 0), new MyPojo3(0, 0));
-
-		List<MyPojo3> minByLastExpected = ImmutableList.of(
-				new MyPojo3(0, 0), new MyPojo3(0, 0), new MyPojo3(0, 0),
-				new MyPojo3(0, 3), new MyPojo3(0, 3), new MyPojo3(0, 3),
-				new MyPojo3(0, 6), new MyPojo3(0, 6), new MyPojo3(0, 6));
-
-		// some necessary boiler plate
-		TypeInformation<MyPojo3> typeInfo = TypeExtractor.getForObject(new MyPojo3(0, 0));
-
-		ExecutionConfig config = new ExecutionConfig();
-
-		KeySelector<MyPojo3, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
-				new Keys.ExpressionKeys<>(new String[]{"f0"}, typeInfo),
-				typeInfo, config);
-		TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
-
-		// aggregations tested
-		ReduceFunction<MyPojo3> maxByFunctionFirst =
-				new ComparableAggregator<>("f1", typeInfo, AggregationType.MAXBY, true, config);
-		ReduceFunction<MyPojo3> maxByFunctionLast =
-				new ComparableAggregator<>("f1", typeInfo, AggregationType.MAXBY, false, config);
-		ReduceFunction<MyPojo3> minByFunctionFirst =
-				new ComparableAggregator<>("f1", typeInfo, AggregationType.MINBY, true, config);
-		ReduceFunction<MyPojo3> minByFunctionLast =
-				new ComparableAggregator<>("f1", typeInfo, AggregationType.MINBY, false, config);
-
-		assertEquals(maxByFirstExpected, MockContext.createAndExecuteForKeyedStream(
-						new StreamGroupedReduce<>(maxByFunctionFirst, typeInfo.createSerializer(config)),
-						getInputByPojoList(),
-						keySelector, keyType));
-		
-		assertEquals(maxByLastExpected, MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(maxByFunctionLast, typeInfo.createSerializer(config)),
-				getInputByPojoList(),
-				keySelector, keyType));
-		
-		assertEquals(minByLastExpected, MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(minByFunctionLast, typeInfo.createSerializer(config)),
-				getInputByPojoList(),
-				keySelector, keyType));
-
-		assertEquals(minByFirstExpected, MockContext.createAndExecuteForKeyedStream(
-				new StreamGroupedReduce<>(minByFunctionFirst, typeInfo.createSerializer(config)),
-				getInputByPojoList(),
-				keySelector, keyType));
-	}
-
-	// *************************************************************************
-	//     UTILS
-	// *************************************************************************
-
-	private List<Tuple2<Integer, Integer>> getInputList() {
-		ArrayList<Tuple2<Integer, Integer>> inputList = new ArrayList<>();
-		for (int i = 0; i < 9; i++) {
-			inputList.add(Tuple2.of(i % 3, i));
-		}
-		return inputList;
-	}
-
-	private List<MyPojo> getInputPojoList() {
-		ArrayList<MyPojo> inputList = new ArrayList<>();
-		for (int i = 0; i < 9; i++) {
-			inputList.add(new MyPojo(i % 3, i));
-		}
-		return inputList;
-	}
-
-	private List<Tuple3<Integer, Integer, Integer>> getInputByList() {
-		ArrayList<Tuple3<Integer, Integer, Integer>> inputList = new ArrayList<>();
-		for (int i = 0; i < 9; i++) {
-			inputList.add(Tuple3.of(0, i % 3, i));
-		}
-		return inputList;
-	}
-
-	private List<MyPojo3> getInputByPojoList() {
-		ArrayList<MyPojo3> inputList = new ArrayList<>();
-		for (int i = 0; i < 9; i++) {
-			inputList.add(new MyPojo3(i % 3, i));
-		}
-		return inputList;
-	}
-
-	public static class MyPojo implements Serializable {
-		
-		private static final long serialVersionUID = 1L;
-		public int f0;
-		public int f1;
-
-		public MyPojo(int f0, int f1) {
-			this.f0 = f0;
-			this.f1 = f1;
-		}
-
-		public MyPojo() {
-		}
-
-		@Override
-		public String toString() {
-			return "POJO(" + f0 + "," + f1 + ")";
-		}
-
-		@Override
-		public boolean equals(Object other) {
-			if (other instanceof MyPojo) {
-				return this.f0 == ((MyPojo) other).f0 && this.f1 == ((MyPojo) other).f1;
-			} else {
-				return false;
-			}
-		}
-	}
-
-	public static class MyPojo3 implements Serializable {
-
-		private static final long serialVersionUID = 1L;
-		public int f0;
-		public int f1;
-		public int f2;
-
-		// Field 0 is always initialized to 0
-		public MyPojo3(int f1, int f2) {
-			this.f1 = f1;
-			this.f2 = f2;
-		}
-
-		public MyPojo3() {
-		}
-
-		@Override
-		public String toString() {
-			return "POJO3(" + f0 + "," + f1 + "," + f2 + ")";
-		}
-
-		@Override
-		public boolean equals(Object other) {
-			if (other instanceof MyPojo3) {
-				return this.f0 == ((MyPojo3) other).f0
-						&& this.f1 == ((MyPojo3) other).f1
-						&& this.f2 == ((MyPojo3) other).f2;
-			} else {
-				return false;
-			}
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
deleted file mode 100644
index 68a047c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
+++ /dev/null
@@ -1,80 +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;
-
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.util.NoOpSink;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-
-import org.junit.Test;
-
-@SuppressWarnings("serial")
-public class ChainedRuntimeContextTest extends StreamingMultipleProgramsTestBase {
-	private static RuntimeContext srcContext;
-	private static RuntimeContext mapContext;
-
-	@Test
-	public void test() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		env.addSource(new TestSource()).map(new TestMap()).addSink(new NoOpSink<Integer>());
-		env.execute();
-
-		assertNotEquals(srcContext, mapContext);
-
-	}
-
-	private static class TestSource extends RichParallelSourceFunction<Integer> {
-
-		@Override
-		public void run(SourceContext<Integer> ctx) throws Exception {
-		}
-
-		@Override
-		public void cancel() {
-		}
-
-		@Override
-		public void open(Configuration c) {
-			srcContext = getRuntimeContext();
-		}
-
-	}
-
-	private static class TestMap extends RichMapFunction<Integer, Integer> {
-
-		@Override
-		public Integer map(Integer value) throws Exception {
-			return value;
-		}
-
-		@Override
-		public void open(Configuration c) {
-			mapContext = getRuntimeContext();
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
deleted file mode 100644
index 0f9cbe9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoStreamTest.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-public class CoStreamTest extends StreamingMultipleProgramsTestBase {
-
-	private static ArrayList<String> expected = new ArrayList<String>();
-
-	@Test
-	public void test() {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		TestListResultSink<String> resultSink = new TestListResultSink<String>();
-
-		DataStream<Integer> src = env.fromElements(1, 3, 5);
-
-		DataStream<Integer> filter1 = src.filter(new FilterFunction<Integer>() {
-	
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public boolean filter(Integer value) throws Exception {
-				return true;
-			}
-		}).keyBy(new KeySelector<Integer, Integer>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Integer getKey(Integer value) throws Exception {
-				return value;
-			}
-		});
-
-		DataStream<Tuple2<Integer, Integer>> filter2 = src
-				.map(new MapFunction<Integer, Tuple2<Integer, Integer>>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public Tuple2<Integer, Integer> map(Integer value) throws Exception {
-						return new Tuple2<Integer, Integer>(value, value + 1);
-					}
-				})
-				.rebalance()
-				.filter(new FilterFunction<Tuple2<Integer, Integer>>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public boolean filter(Tuple2<Integer, Integer> value) throws Exception {
-						return true;
-					}
-				}).disableChaining().keyBy(new KeySelector<Tuple2<Integer, Integer>, Integer>() {
-
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public Integer getKey(Tuple2<Integer, Integer> value) throws Exception {
-						return value.f0;
-					}
-				});
-
-		DataStream<String> connected = filter1.connect(filter2).flatMap(new CoFlatMapFunction<Integer, Tuple2<Integer, Integer>, String>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void flatMap1(Integer value, Collector<String> out) throws Exception {
-				out.collect(value.toString());
-			}
-
-			@Override
-			public void flatMap2(Tuple2<Integer, Integer> value, Collector<String> out) throws Exception {
-				out.collect(value.toString());
-			}
-		});
-
-		connected.addSink(resultSink);
-
-		try {
-			env.execute();
-		} catch (Exception e) {
-			e.printStackTrace();
-		}
-
-		expected = new ArrayList<String>();
-		expected.addAll(Arrays.asList("(1,2)", "(3,4)", "(5,6)", "1", "3", "5"));
-
-		List<String> result = resultSink.getResult();
-		Collections.sort(result);
-
-		assertEquals(expected, result);
-	}
-}


[04/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
new file mode 100644
index 0000000..f25c995
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.collector.selector;
+
+import java.io.Serializable;
+
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Collector;
+
+public interface OutputSelectorWrapper<OUT> extends Serializable {
+
+	public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge);
+
+	public Iterable<Collector<StreamRecord<OUT>>> getSelectedOutputs(OUT record);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java
new file mode 100644
index 0000000..dca2ede
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapperFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.collector.selector;
+
+import java.util.List;
+
+public class OutputSelectorWrapperFactory {
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	public static OutputSelectorWrapper<?> create(List<OutputSelector<?>> outputSelectors) {
+		if (outputSelectors.size() == 0) {
+			return new BroadcastOutputSelectorWrapper();
+		} else {
+			return new DirectedOutputSelectorWrapper(outputSelectors);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
new file mode 100644
index 0000000..7191304
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -0,0 +1,556 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.FoldFunction;
+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.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
+import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
+import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
+import org.apache.flink.streaming.api.functions.windowing.FoldAllWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+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 AllWindowedStream} 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 AllWindowedStream} is purely and API construct, during runtime
+ * the {@code AllWindowedStream} 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 AllWindowedStream<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 AllWindowedStream(DataStream<T> input,
+			WindowAssigner<? super T, W> windowAssigner) {
+		this.input = input;
+		this.windowAssigner = windowAssigner;
+		this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
+	}
+
+	/**
+	 * Sets the {@code Trigger} that should be used to trigger window emission.
+	 */
+	public AllWindowedStream<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 AllWindowedStream<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 SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "Reduce at " + callLocation;
+
+		SingleOutputStreamOperator<T, ?> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
+		if (result != null) {
+			return result;
+		}
+
+		String opName = "NonParallelTriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+
+		OneInputStreamOperator<T, T> operator;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
+		if (evictor != null) {
+			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					new ReduceAllWindowFunction<W, T>(function),
+					trigger,
+					evictor).enableSetProcessingTime(setProcessingTime);
+
+		} 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,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
+					new ReduceAllWindowFunction<W, T>(function),
+					trigger).enableSetProcessingTime(setProcessingTime);
+		}
+
+		return input.transform(opName, input.getType(), operator).setParallelism(1);
+	}
+
+	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
+				Utils.getCallLocationName(), true);
+
+		return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
+	}
+
+	/**
+	 * Applies the given fold function to each 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.
+	 *
+	 * @param function The fold function.
+	 * @return The data stream that is the result of applying the fold function to the window.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+		return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
+	}
+
+	/**
+	 * 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> SingleOutputStreamOperator<R, ?> apply(AllWindowFunction<T, R, W> function) {
+		TypeInformation<T> inType = input.getType();
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
+				function, AllWindowFunction.class, true, true, inType, null, false);
+
+		return apply(function, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each evaluation
+	 * of the window for each key individually. The output of the window function is interpreted
+	 * as a regular non-windowed stream.
+	 * <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> SingleOutputStreamOperator<R, ?> apply(AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
+		//clean the closure
+		function = input.getExecutionEnvironment().clean(function);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "WindowApply at " + callLocation;
+
+		SingleOutputStreamOperator<R, ?> result = createFastTimeOperatorIfValid(function, resultType, udfName);
+		if (result != null) {
+			return result;
+		}
+
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+
+		NonKeyedWindowOperator<T, R, W> operator;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
+		if (evictor != null) {
+			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor).enableSetProcessingTime(setProcessingTime);
+
+		} else {
+			operator = new NonKeyedWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger).enableSetProcessingTime(setProcessingTime);
+		}
+
+		return input.transform(opName, resultType, operator).setParallelism(1);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>
+	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
+	 *
+	 * @param preAggregator The reduce function that is used for 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> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, AllWindowFunction<T, R, W> function) {
+		TypeInformation<T> inType = input.getType();
+		TypeInformation<R> resultType = TypeExtractor.getUnaryOperatorReturnType(
+				function, AllWindowFunction.class, true, true, inType, null, false);
+
+		return apply(preAggregator, function, resultType);
+	}
+
+	/**
+	 * Applies the given window function to each window. The window function is called for each
+	 * evaluation of the window for each key individually. The output of the window function is
+	 * interpreted as a regular non-windowed stream.
+	 *
+	 * <p>
+	 * Arriving data is pre-aggregated using the given pre-aggregation reducer.
+	 *
+	 * @param preAggregator The reduce function that is used for pre-aggregation
+	 * @param function The window function.
+	 * @param resultType Type information for the result type of the window function
+	 * @return The data stream that is the result of applying the window function to the window.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> apply(ReduceFunction<T> preAggregator, AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) {
+		//clean the closures
+		function = input.getExecutionEnvironment().clean(function);
+		preAggregator = input.getExecutionEnvironment().clean(preAggregator);
+
+		String callLocation = Utils.getCallLocationName();
+		String udfName = "WindowApply at " + callLocation;
+
+		String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
+
+		OneInputStreamOperator<T, R> operator;
+
+		boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
+		if (evictor != null) {
+			operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					new HeapWindowBuffer.Factory<T>(),
+					function,
+					trigger,
+					evictor).enableSetProcessingTime(setProcessingTime);
+
+		} else {
+			operator = new NonKeyedWindowOperator<>(windowAssigner,
+					windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
+					new PreAggregatingHeapWindowBuffer.Factory<>(preAggregator),
+					function,
+					trigger).enableSetProcessingTime(setProcessingTime);
+		}
+
+		return input.transform(opName, resultType, operator).setParallelism(1);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Aggregations on the  windows
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Applies an aggregation that sums every window of the data stream at the
+	 * given position.
+	 *
+	 * @param positionToSum The position in the tuple/array to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
+		return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that sums every window of the pojo data stream at
+	 * the given field for every window.
+	 *
+	 * <p>
+	 * A field expression is either
+	 * the name of a public field or a getter method with parentheses of the
+	 * stream's underlying type. A dot can be used to drill down into objects,
+	 * as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field to sum
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> sum(String field) {
+		return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of every window
+	 * of the data stream at the given position.
+	 *
+	 * @param positionToMin The position to minimize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
+		return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum value of the pojo data
+	 * stream at the given field expression for every window.
+	 *
+	 * <p>
+	 * A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> min(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy
+	 *            The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns the first element by default.
+	 *
+	 * @param positionToMinBy The position to minimize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
+		return this.minBy(positionToMinBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the minimum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * minimum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMinBy The position to minimize
+	 * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the minimum element of the pojo
+	 * data stream by the given field expression for every window. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream DataStreams} underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @param first If True then in case of field equality the first object will be returned
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum value of every window of
+	 * the data stream at the given position.
+	 *
+	 * @param positionToMax The position to maximize
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
+		return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum value of the pojo data
+	 * stream at the given field expression for every window. A field expression
+	 * is either the name of a public field or a getter method with parentheses
+	 * of the {@link DataStream DataStreams} underlying type. A dot can be used to drill
+	 * down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> max(String field) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns the first by default.
+	 *
+	 * @param positionToMaxBy
+	 *            The position to maximize by
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
+		return this.maxBy(positionToMaxBy, true);
+	}
+
+	/**
+	 * Applies an aggregation that gives the maximum element of every window of
+	 * the data stream by the given position. If more elements have the same
+	 * maximum value the operator returns either the first or last one depending
+	 * on the parameter setting.
+	 *
+	 * @param positionToMaxBy The position to maximize by
+	 * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
+		return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	/**
+	 * Applies an aggregation that that gives the maximum element of the pojo
+	 * data stream by the given field expression for every window. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field The field expression based on which the aggregation will be applied.
+	 * @param first If True then in case of field equality the first object will be returned
+	 * @return The transformed DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
+		return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+	}
+
+	private SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregator) {
+		return reduce(aggregator);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+
+	private <R> SingleOutputStreamOperator<R, ?> createFastTimeOperatorIfValid(
+			Function function,
+			TypeInformation<R> resultType,
+			String functionName) {
+
+		// TODO: add once non-parallel fast aligned time windows operator is ready
+		return null;
+	}
+
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return input.getExecutionEnvironment();
+	}
+
+	public TypeInformation<T> getInputType() {
+		return input.getType();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
new file mode 100644
index 0000000..d1da783
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
@@ -0,0 +1,575 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+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.util.Collector;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ *{@code CoGroupedStreams} represents two {@link DataStream DataStreams} that have been co-grouped.
+ * A streaming co-group operation is evaluated over elements in a window.
+ *
+ * <p>
+ * To finalize co-group operation you also need to specify a {@link KeySelector} for
+ * both the first and second input and a {@link WindowAssigner}.
+ *
+ * <p>
+ * Note: Right now, the groups are being built in memory so you need to ensure that they don't
+ * get too big. Otherwise the JVM might crash.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> one = ...;
+ * DataStream<Tuple2<String, Integer>> two = ...;
+ *
+ * DataStream<T> result = one.coGroup(two)
+ *     .where(new MyFirstKeySelector())
+ *     .equalTo(new MyFirstKeySelector())
+ *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ *     .apply(new MyCoGroupFunction());
+ * } </pre>
+ */
+public class CoGroupedStreams<T1, T2> {
+
+	/** The first input stream */
+	private final DataStream<T1> input1;
+
+	/** The second input stream */
+	private final DataStream<T2> input2;
+
+	/**
+	 * Creates new CoGroped data streams, which are the first step towards building a streaming co-group.
+	 * 
+	 * @param input1 The first data stream.
+	 * @param input2 The second data stream.
+	 */
+	public CoGroupedStreams(DataStream<T1> input1, DataStream<T2> input2) {
+		this.input1 = requireNonNull(input1);
+		this.input2 = requireNonNull(input2);
+	}
+
+	/**
+	 * Specifies a {@link KeySelector} for elements from the first input.
+	 */
+	public <KEY> Where<KEY> where(KeySelector<T1, KEY> keySelector)  {
+		TypeInformation<KEY> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+		return new Where<>(input1.clean(keySelector), keyType);
+	}
+
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * CoGrouped streams that have the key for one side defined.
+	 * 
+	 * @param <KEY> The type of the key.
+	 */
+	public class Where<KEY> {
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final TypeInformation<KEY> keyType;
+
+		Where(KeySelector<T1, KEY> keySelector1, TypeInformation<KEY> keyType) {
+			this.keySelector1 = keySelector1;
+			this.keyType = keyType;
+		}
+	
+		/**
+		 * Specifies a {@link KeySelector} for elements from the second input.
+		 */
+		public EqualTo equalTo(KeySelector<T2, KEY> keySelector)  {
+			TypeInformation<KEY> otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+			if (!otherKey.equals(this.keyType)) {
+				throw new IllegalArgumentException("The keys for the two inputs are not equal: " + 
+						"first key = " + this.keyType + " , second key = " + otherKey);
+			}
+			
+			return new EqualTo(input2.clean(keySelector));
+		}
+
+		// --------------------------------------------------------------------
+		
+		/**
+		 * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs.
+		 */
+		public class EqualTo {
+
+			private final KeySelector<T2, KEY> keySelector2;
+
+			EqualTo(KeySelector<T2, KEY> keySelector2) {
+				this.keySelector2 = requireNonNull(keySelector2);
+			}
+
+			/**
+			 * Specifies the window on which the co-group operation works.
+			 */
+			public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+				return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs as
+	 * well as a {@link WindowAssigner}.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 * @param <KEY> Type of the key. This must be the same for both inputs
+	 * @param <W> Type of {@link Window} on which the co-group operation works.
+	 */
+	public static class WithWindow<T1, T2, KEY, W extends Window> {
+		private final DataStream<T1> input1;
+		private final DataStream<T2> input2;
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final KeySelector<T2, KEY> keySelector2;
+		
+		private final TypeInformation<KEY> keyType;
+
+		private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
+
+		private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
+
+		private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
+
+		protected WithWindow(DataStream<T1> input1,
+				DataStream<T2> input2,
+				KeySelector<T1, KEY> keySelector1,
+				KeySelector<T2, KEY> keySelector2,
+				TypeInformation<KEY> keyType,
+				WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
+				Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
+				Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
+			this.input1 = input1;
+			this.input2 = input2;
+
+			this.keySelector1 = keySelector1;
+			this.keySelector2 = keySelector2;
+			this.keyType = keyType;
+			
+			this.windowAssigner = windowAssigner;
+			this.trigger = trigger;
+			this.evictor = evictor;
+		}
+
+		/**
+		 * Sets the {@code Trigger} that should be used to trigger window emission.
+		 */
+		public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+					windowAssigner, newTrigger, evictor);
+		}
+
+		/**
+		 * 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 WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+					windowAssigner, trigger, newEvictor);
+		}
+
+		/**
+		 * Completes the co-group operation with the user function that is executed
+		 * for windowed groups.
+		 */
+		public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function) {
+
+			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+					function,
+					CoGroupFunction.class,
+					true,
+					true,
+					input1.getType(),
+					input2.getType(),
+					"CoGroup",
+					false);
+
+			return apply(function, resultType);
+		}
+
+		/**
+		 * Completes the co-group operation with the user function that is executed
+		 * for windowed groups.
+		 */
+		public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+			//clean the closure
+			function = input1.getExecutionEnvironment().clean(function);
+
+			UnionTypeInfo<T1, T2> unionType = new UnionTypeInfo<>(input1.getType(), input2.getType());
+			UnionKeySelector<T1, T2, KEY> unionKeySelector = new UnionKeySelector<>(keySelector1, keySelector2);
+			
+			DataStream<TaggedUnion<T1, T2>> taggedInput1 = input1
+					.map(new Input1Tagger<T1, T2>())
+					.returns(unionType);
+			DataStream<TaggedUnion<T1, T2>> taggedInput2 = input2
+					.map(new Input2Tagger<T1, T2>())
+					.returns(unionType);
+
+			DataStream<TaggedUnion<T1, T2>> unionStream = taggedInput1.union(taggedInput2);
+			
+			// we explicitly create the keyed stream to manually pass the key type information in
+			WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp = 
+					new KeyedStream<TaggedUnion<T1, T2>, KEY>(unionStream, unionKeySelector, keyType)
+					.window(windowAssigner);
+
+			if (trigger != null) {
+				windowOp.trigger(trigger);
+			}
+			if (evictor != null) {
+				windowOp.evictor(evictor);
+			}
+
+			return windowOp.apply(new CoGroupWindowFunction<T1, T2, T, KEY, W>(function), resultType);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Data type and type information for Tagged Union
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Internal class for implementing tagged union co-group.
+	 */
+	public static class TaggedUnion<T1, T2> {
+		private final T1 one;
+		private final T2 two;
+
+		private TaggedUnion(T1 one, T2 two) {
+			this.one = one;
+			this.two = two;
+		}
+
+		public boolean isOne() {
+			return one != null;
+		}
+
+		public boolean isTwo() {
+			return two != null;
+		}
+
+		public T1 getOne() {
+			return one;
+		}
+
+		public T2 getTwo() {
+			return two;
+		}
+
+		public static <T1, T2> TaggedUnion<T1, T2> one(T1 one) {
+			return new TaggedUnion<>(one, null);
+		}
+
+		public static <T1, T2> TaggedUnion<T1, T2> two(T2 two) {
+			return new TaggedUnion<>(null, two);
+		}
+	}
+
+	private static class UnionTypeInfo<T1, T2> extends TypeInformation<TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		TypeInformation<T1> oneType;
+		TypeInformation<T2> twoType;
+
+		public UnionTypeInfo(TypeInformation<T1> oneType,
+				TypeInformation<T2> twoType) {
+			this.oneType = oneType;
+			this.twoType = twoType;
+		}
+
+		@Override
+		public boolean isBasicType() {
+			return false;
+		}
+
+		@Override
+		public boolean isTupleType() {
+			return false;
+		}
+
+		@Override
+		public int getArity() {
+			return 2;
+		}
+
+		@Override
+		public int getTotalFields() {
+			return 2;
+		}
+
+		@Override
+		@SuppressWarnings("unchecked, rawtypes")
+		public Class<TaggedUnion<T1, T2>> getTypeClass() {
+			return (Class) TaggedUnion.class;
+		}
+
+		@Override
+		public boolean isKeyType() {
+			return true;
+		}
+
+		@Override
+		public TypeSerializer<TaggedUnion<T1, T2>> createSerializer(ExecutionConfig config) {
+			return new UnionSerializer<>(oneType.createSerializer(config), twoType.createSerializer(config));
+		}
+
+		@Override
+		public String toString() {
+			return "TaggedUnion<" + oneType + ", " + twoType + ">";
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			if (obj instanceof UnionTypeInfo) {
+				@SuppressWarnings("unchecked")
+				UnionTypeInfo<T1, T2> unionTypeInfo = (UnionTypeInfo<T1, T2>) obj;
+
+				return unionTypeInfo.canEqual(this) && oneType.equals(unionTypeInfo.oneType) && twoType.equals(unionTypeInfo.twoType);
+			} else {
+				return false;
+			}
+		}
+
+		@Override
+		public int hashCode() {
+			return 31 *  oneType.hashCode() + twoType.hashCode();
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj instanceof UnionTypeInfo;
+		}
+	}
+
+	private static class UnionSerializer<T1, T2> extends TypeSerializer<TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		private final TypeSerializer<T1> oneSerializer;
+		private final TypeSerializer<T2> twoSerializer;
+
+		public UnionSerializer(TypeSerializer<T1> oneSerializer,
+				TypeSerializer<T2> twoSerializer) {
+			this.oneSerializer = oneSerializer;
+			this.twoSerializer = twoSerializer;
+		}
+
+		@Override
+		public boolean isImmutableType() {
+			return false;
+		}
+
+		@Override
+		public TypeSerializer<TaggedUnion<T1, T2>> duplicate() {
+			return this;
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> createInstance() {
+			return null;
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from) {
+			if (from.isOne()) {
+				return TaggedUnion.one(oneSerializer.copy(from.getOne()));
+			} else {
+				return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
+			}
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from, TaggedUnion<T1, T2> reuse) {
+			if (from.isOne()) {
+				return TaggedUnion.one(oneSerializer.copy(from.getOne()));
+			} else {
+				return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
+			}		}
+
+		@Override
+		public int getLength() {
+			return -1;
+		}
+
+		@Override
+		public void serialize(TaggedUnion<T1, T2> record, DataOutputView target) throws IOException {
+			if (record.isOne()) {
+				target.writeByte(1);
+				oneSerializer.serialize(record.getOne(), target);
+			} else {
+				target.writeByte(2);
+				twoSerializer.serialize(record.getTwo(), target);
+			}
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> deserialize(DataInputView source) throws IOException {
+			byte tag = source.readByte();
+			if (tag == 1) {
+				return TaggedUnion.one(oneSerializer.deserialize(source));
+			} else {
+				return TaggedUnion.two(twoSerializer.deserialize(source));
+			}
+		}
+
+		@Override
+		public TaggedUnion<T1, T2> deserialize(TaggedUnion<T1, T2> reuse,
+				DataInputView source) throws IOException {
+			byte tag = source.readByte();
+			if (tag == 1) {
+				return TaggedUnion.one(oneSerializer.deserialize(source));
+			} else {
+				return TaggedUnion.two(twoSerializer.deserialize(source));
+			}
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			byte tag = source.readByte();
+			target.writeByte(tag);
+			if (tag == 1) {
+				oneSerializer.copy(source, target);
+			} else {
+				twoSerializer.copy(source, target);
+			}
+		}
+
+		@Override
+		public int hashCode() {
+			return 31 * oneSerializer.hashCode() + twoSerializer.hashCode();
+		}
+
+		@Override
+		@SuppressWarnings("unchecked")
+		public boolean equals(Object obj) {
+			if (obj instanceof UnionSerializer) {
+				UnionSerializer<T1, T2> other = (UnionSerializer<T1, T2>) obj;
+
+				return other.canEqual(this) && oneSerializer.equals(other.oneSerializer) && twoSerializer.equals(other.twoSerializer);
+			} else {
+				return false;
+			}
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj instanceof UnionSerializer;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utility functions that implement the CoGroup logic based on the tagged
+	//  untion window reduce
+	// ------------------------------------------------------------------------
+	
+	private static class Input1Tagger<T1, T2> implements MapFunction<T1, TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public TaggedUnion<T1, T2> map(T1 value) throws Exception {
+			return TaggedUnion.one(value);
+		}
+	}
+
+	private static class Input2Tagger<T1, T2> implements MapFunction<T2, TaggedUnion<T1, T2>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public TaggedUnion<T1, T2> map(T2 value) throws Exception {
+			return TaggedUnion.two(value);
+		}
+	}
+
+	private static class UnionKeySelector<T1, T2, KEY> implements KeySelector<TaggedUnion<T1, T2>, KEY> {
+		private static final long serialVersionUID = 1L;
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final KeySelector<T2, KEY> keySelector2;
+
+		public UnionKeySelector(KeySelector<T1, KEY> keySelector1,
+				KeySelector<T2, KEY> keySelector2) {
+			this.keySelector1 = keySelector1;
+			this.keySelector2 = keySelector2;
+		}
+
+		@Override
+		public KEY getKey(TaggedUnion<T1, T2> value) throws Exception{
+			if (value.isOne()) {
+				return keySelector1.getKey(value.getOne());
+			} else {
+				return keySelector2.getKey(value.getTwo());
+			}
+		}
+	}
+
+	private static class CoGroupWindowFunction<T1, T2, T, KEY, W extends Window>
+			extends WrappingFunction<CoGroupFunction<T1, T2, T>>
+			implements WindowFunction<TaggedUnion<T1, T2>, T, KEY, W> {
+		
+		private static final long serialVersionUID = 1L;
+
+		public CoGroupWindowFunction(CoGroupFunction<T1, T2, T> userFunction) {
+			super(userFunction);
+		}
+
+		@Override
+		public void apply(KEY key,
+				W window,
+				Iterable<TaggedUnion<T1, T2>> values,
+				Collector<T> out) throws Exception {
+			
+			List<T1> oneValues = new ArrayList<>();
+			List<T2> twoValues = new ArrayList<>();
+			
+			for (TaggedUnion<T1, T2> val: values) {
+				if (val.isOne()) {
+					oneValues.add(val.getOne());
+				} else {
+					twoValues.add(val.getTwo());
+				}
+			}
+			wrappedFunction.coGroup(oneValues, twoValues, out);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
new file mode 100644
index 0000000..4074a1d
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java
@@ -0,0 +1,331 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.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.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
+import org.apache.flink.streaming.api.functions.co.CoMapFunction;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.operators.co.CoStreamFlatMap;
+import org.apache.flink.streaming.api.operators.co.CoStreamMap;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+
+/**
+ * {@code ConnectedStreams} represents two connected streams of (possible) different data types. It
+ * can be used to apply transformations such as {@link CoMapFunction} on two
+ * {@link DataStream DataStreams}
+ * 
+ * @param <IN1> Type of the first input data steam.
+ * @param <IN2> Type of the second input data stream.
+ */
+public class ConnectedStreams<IN1, IN2> {
+
+	protected StreamExecutionEnvironment environment;
+	protected DataStream<IN1> inputStream1;
+	protected DataStream<IN2> inputStream2;
+
+	protected ConnectedStreams(StreamExecutionEnvironment env,
+			DataStream<IN1> input1,
+			DataStream<IN2> input2) {
+		this.environment = env;
+		if (input1 != null) {
+			this.inputStream1 = input1;
+		}
+		if (input2 != null) {
+			this.inputStream2 = input2;
+		}
+	}
+
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return environment;
+	}
+
+	/**
+	 * Returns the first {@link DataStream}.
+	 *
+	 * @return The first DataStream.
+	 */
+	public DataStream<IN1> getFirstInput() {
+		return inputStream1;
+	}
+
+	/**
+	 * Returns the second {@link DataStream}.
+	 *
+	 * @return The second DataStream.
+	 */
+	public DataStream<IN2> getSecondInput() {
+		return inputStream2;
+	}
+
+	/**
+	 * Gets the type of the first input
+	 *
+	 * @return The type of the first input
+	 */
+	public TypeInformation<IN1> getType1() {
+		return inputStream1.getType();
+	}
+
+	/**
+	 * Gets the type of the second input
+	 *
+	 * @return The type of the second input
+	 */
+	public TypeInformation<IN2> getType2() {
+		return inputStream2.getType();
+	}
+
+	/**
+	 * KeyBy operation for connected data stream. Assigns keys to the elements of
+	 * input1 and input2 according to keyPosition1 and keyPosition2.
+	 *
+	 * @param keyPosition1
+	 *            The field used to compute the hashcode of the elements in the
+	 *            first input stream.
+	 * @param keyPosition2
+	 *            The field used to compute the hashcode of the elements in the
+	 *            second input stream.
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> keyBy(int keyPosition1, int keyPosition2) {
+		return new ConnectedStreams<>(this.environment, inputStream1.keyBy(keyPosition1),
+				inputStream2.keyBy(keyPosition2));
+	}
+
+	/**
+	 * KeyBy operation for connected data stream. Assigns keys to the elements of
+	 * input1 and input2 according to keyPositions1 and keyPositions2.
+	 *
+	 * @param keyPositions1
+	 *            The fields used to group the first input stream.
+	 * @param keyPositions2
+	 *            The fields used to group the second input stream.
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> keyBy(int[] keyPositions1, int[] keyPositions2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(keyPositions1),
+				inputStream2.keyBy(keyPositions2));
+	}
+
+	/**
+	 * KeyBy operation for connected data stream using key expressions. Assigns keys to
+	 * the elements of input1 and input2 according to field1 and field2. A field
+	 * expression is either the name of a public field or a getter method with
+	 * parentheses of the {@link DataStream}S underlying type. A dot can be used
+	 * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param field1
+	 *            The grouping expression for the first input
+	 * @param field2
+	 *            The grouping expression for the second input
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> keyBy(String field1, String field2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(field1),
+				inputStream2.keyBy(field2));
+	}
+
+	/**
+	 * KeyBy operation for connected data stream using key expressions.
+	 * the elements of input1 and input2 according to fields1 and fields2. A
+	 * field expression is either the name of a public field or a getter method
+	 * with parentheses of the {@link DataStream}S underlying type. A dot can be
+	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
+	 * .
+	 *
+	 * @param fields1
+	 *            The grouping expressions for the first input
+	 * @param fields2
+	 *            The grouping expressions for the second input
+	 * @return The grouped {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> keyBy(String[] fields1, String[] fields2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(fields1),
+				inputStream2.keyBy(fields2));
+	}
+
+	/**
+	 * KeyBy operation for connected data stream. Assigns keys to the elements of
+	 * input1 and input2 using keySelector1 and keySelector2.
+	 *
+	 * @param keySelector1
+	 *            The {@link KeySelector} used for grouping the first input
+	 * @param keySelector2
+	 *            The {@link KeySelector} used for grouping the second input
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> keyBy(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
+		return new ConnectedStreams<>(environment, inputStream1.keyBy(keySelector1),
+				inputStream2.keyBy(keySelector2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream. Partitions the elements of
+	 * input1 and input2 according to keyPosition1 and keyPosition2.
+	 *
+	 * @param keyPosition1
+	 *            The field used to compute the hashcode of the elements in the
+	 *            first input stream.
+	 * @param keyPosition2
+	 *            The field used to compute the hashcode of the elements in the
+	 *            second input stream.
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(int keyPosition1, int keyPosition2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keyPosition1),
+				inputStream2.partitionByHash(keyPosition2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream. Partitions the elements of
+	 * input1 and input2 according to keyPositions1 and keyPositions2.
+	 *
+	 * @param keyPositions1
+	 *            The fields used to group the first input stream.
+	 * @param keyPositions2
+	 *            The fields used to group the second input stream.
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(int[] keyPositions1, int[] keyPositions2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keyPositions1),
+				inputStream2.partitionByHash(keyPositions2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream using key expressions. Partitions
+	 * the elements of input1 and input2 according to field1 and field2. A
+	 * field expression is either the name of a public field or a getter method
+	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
+	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
+	 *
+	 * @param field1
+	 *            The partitioning expressions for the first input
+	 * @param field2
+	 *            The partitioning expressions for the second input
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(String field1, String field2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(field1),
+				inputStream2.partitionByHash(field2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream using key expressions. Partitions
+	 * the elements of input1 and input2 according to fields1 and fields2. A
+	 * field expression is either the name of a public field or a getter method
+	 * with parentheses of the {@link DataStream}s underlying type. A dot can be
+	 * used to drill down into objects, as in {@code "field1.getInnerField2()" }
+	 *
+	 * @param fields1
+	 *            The partitioning expressions for the first input
+	 * @param fields2
+	 *            The partitioning expressions for the second input
+	 * @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(String[] fields1, String[] fields2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(fields1),
+				inputStream2.partitionByHash(fields2));
+	}
+
+	/**
+	 * PartitionBy operation for connected data stream. Partitions the elements of
+	 * input1 and input2 using keySelector1 and keySelector2.
+	 *
+	 * @param keySelector1
+	 *            The {@link KeySelector} used for partitioning the first input
+	 * @param keySelector2
+	 *            The {@link KeySelector} used for partitioning the second input
+	 * @return @return The partitioned {@link ConnectedStreams}
+	 */
+	public ConnectedStreams<IN1, IN2> partitionByHash(KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) {
+		return new ConnectedStreams<>(environment, inputStream1.partitionByHash(keySelector1),
+				inputStream2.partitionByHash(keySelector2));
+	}
+
+	/**
+	 * Applies a CoMap transformation on a {@link ConnectedStreams} and maps
+	 * the output to a common type. The transformation calls a
+	 * {@link CoMapFunction#map1} for each element of the first input and
+	 * {@link CoMapFunction#map2} for each element of the second input. Each
+	 * CoMapFunction call returns exactly one element.
+	 * 
+	 * @param coMapper The CoMapFunction used to jointly transform the two input DataStreams
+	 * @return The transformed {@link DataStream}
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> map(CoMapFunction<IN1, IN2, OUT> coMapper) {
+
+		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coMapper,
+				CoMapFunction.class, false, true, getType1(), getType2(),
+				Utils.getCallLocationName(), true);
+
+		return transform("Co-Map", outTypeInfo, new CoStreamMap<>(inputStream1.clean(coMapper)));
+
+	}
+
+	/**
+	 * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and
+	 * maps the output to a common type. The transformation calls a
+	 * {@link CoFlatMapFunction#flatMap1} for each element of the first input
+	 * and {@link CoFlatMapFunction#flatMap2} for each element of the second
+	 * input. Each CoFlatMapFunction call returns any number of elements
+	 * including none.
+	 * 
+	 * @param coFlatMapper
+	 *            The CoFlatMapFunction used to jointly transform the two input
+	 *            DataStreams
+	 * @return The transformed {@link DataStream}
+	 */
+	public <OUT> SingleOutputStreamOperator<OUT, ?> flatMap(
+			CoFlatMapFunction<IN1, IN2, OUT> coFlatMapper) {
+
+		TypeInformation<OUT> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType(coFlatMapper,
+				CoFlatMapFunction.class, false, true, getType1(), getType2(),
+				Utils.getCallLocationName(), true);
+
+		return transform("Co-Flat Map", outTypeInfo, new CoStreamFlatMap<>(inputStream1.clean(coFlatMapper)));
+	}
+
+	public <OUT> SingleOutputStreamOperator<OUT, ?> transform(String functionName,
+			TypeInformation<OUT> outTypeInfo,
+			TwoInputStreamOperator<IN1, IN2, OUT> operator) {
+
+		// read the output type of the input Transforms to coax out errors about MissingTypeInfo
+		inputStream1.getType();
+		inputStream2.getType();
+
+		TwoInputTransformation<IN1, IN2, OUT> transform = new TwoInputTransformation<>(
+				inputStream1.getTransformation(),
+				inputStream2.getTransformation(),
+				functionName,
+				operator,
+				outTypeInfo,
+				environment.getParallelism());
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		SingleOutputStreamOperator<OUT, ?> returnStream = new SingleOutputStreamOperator(environment, transform);
+
+		getExecutionEnvironment().addOperator(transform);
+
+		return returnStream;
+	}
+}


[18/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/pom.xml b/flink-staging/flink-streaming/flink-streaming-examples/pom.xml
deleted file mode 100644
index 563cf01..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/pom.xml
+++ /dev/null
@@ -1,535 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-streaming-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-streaming-examples</artifactId>
-	<name>flink-streaming-examples</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-scala</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java-examples</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-twitter</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-tests</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- get default data from flink-java-examples package -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-dependency-plugin</artifactId>
-				<version>2.9</version><!--$NO-MVN-MAN-VER$-->
-				<executions>
-					<execution>
-						<id>unpack</id>
-						<phase>prepare-package</phase>
-						<goals>
-							<goal>unpack</goal>
-						</goals>
-						<configuration>
-							<artifactItems>
-								<!-- For WordCount example data -->
-								<artifactItem>
-									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-java-examples</artifactId>
-									<version>${project.version}</version>
-									<type>jar</type>
-									<overWrite>false</overWrite>
-									<outputDirectory>${project.build.directory}/classes</outputDirectory>
-									<includes>org/apache/flink/examples/java/wordcount/util/WordCountData.class</includes>
-								</artifactItem>
-								<!-- For JSON utilities -->
-								<artifactItem>
-									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-connector-twitter</artifactId>
-									<version>${project.version}</version>
-									<type>jar</type>
-									<overWrite>false</overWrite>
-									<outputDirectory>${project.build.directory}/classes</outputDirectory>
-									<includes>org/apache/flink/streaming/connectors/json/*</includes>
-								</artifactItem>
-							</artifactItems>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- self-contained jars for each example -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<version>2.4</version><!--$NO-MVN-MAN-VER$-->
-				<executions>
-					<!-- Default Execution -->
-					<execution>
-						<id>default</id>
-						<phase>package</phase>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-					
-					<!-- Iteration -->
-					<execution>
-						<id>Iteration</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>Iteration</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.iteration.IterateExample</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/iteration/*.class</include>			
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- IncrementalLearning -->
-					<execution>
-						<id>IncrementalLearning</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>IncrementalLearning</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.ml.IncrementalLearningSkeleton</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/ml/*.class</include>			
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- Twitter -->
-					<execution>
-						<id>Twitter</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>Twitter</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.twitter.TwitterStream</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/twitter/*.class</include>
-								<include>org/apache/flink/streaming/examples/twitter/util/*.class</include>
-								<include>org/apache/flink/streaming/connectors/json/*.class</include>
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- WindowJoin -->
-					<execution>
-						<id>WindowJoin</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>WindowJoin</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.join.WindowJoin</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/join/*.class</include>			
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- WordCountPOJO -->
-					<execution>
-						<id>WordCountPOJO</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>WordCountPOJO</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.wordcount.PojoExample</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/wordcount/PojoExample.class</include>
-								<include>org/apache/flink/streaming/examples/wordcount/PojoExample$*.class</include>
-								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>			
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- WordCount -->
-					<execution>
-						<id>WordCount</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>WordCount</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.wordcount.WordCount</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/wordcount/WordCount.class</include>
-								<include>org/apache/flink/streaming/examples/wordcount/WordCount$*.class</include>
-								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>				
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- WindowWordCount -->
-					<execution>
-						<id>WindowWordCount</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>WindowWordCount</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.windowing.WindowWordCount</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/windowing/WindowWordCount.class</include>
-								<include>org/apache/flink/streaming/examples/wordcount/WordCount.class</include>
-								<include>org/apache/flink/streaming/examples/wordcount/WordCount$*.class</include>
-								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- SocketTextStreamWordCount -->
-					<execution>
-						<id>SocketTextStreamWordCount</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>SocketTextStreamWordCount</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.socket.SocketTextStreamWordCount</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.class</include>
-								<include>org/apache/flink/streaming/examples/wordcount/WordCount.class</include>
-								<include>org/apache/flink/streaming/examples/wordcount/WordCount$*.class</include>
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- TopSpeedWindowing -->
-					<execution>
-						<id>TopSpeedWindowing</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>TopSpeedWindowing</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.windowing.TopSpeedWindowing</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.class</include>
-								<include>org/apache/flink/streaming/examples/windowing/TopSpeedWindowing$*.class</include>
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- SessionWindowing -->
-					<execution>
-						<id>SessionWindowing</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<classifier>SessionWindowing</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.streaming.examples.windowing.SessionWindowing</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<include>org/apache/flink/streaming/examples/windowing/SessionWindowing.class</include>
-								<include>org/apache/flink/streaming/examples/windowing/SessionWindowing$*.class</include>
-							</includes>
-						</configuration>
-					</execution>
-
-				</executions>
-			</plugin>
-
-
-			<!-- Scala Compiler -->
-			<plugin>
-				<groupId>net.alchim31.maven</groupId>
-				<artifactId>scala-maven-plugin</artifactId>
-				<version>3.1.4</version>
-				<executions>
-					<!-- Run scala compiler in the process-resources phase, so that dependencies on
-						scala classes can be resolved later in the (Java) compile phase -->
-					<execution>
-						<id>scala-compile-first</id>
-						<phase>process-resources</phase>
-						<goals>
-							<goal>compile</goal>
-						</goals>
-					</execution>
- 
-					<!-- Run scala compiler in the process-test-resources phase, so that dependencies on
-						 scala classes can be resolved later in the (Java) test-compile phase -->
-					<execution>
-						<id>scala-test-compile</id>
-						<phase>process-test-resources</phase>
-						<goals>
-							<goal>testCompile</goal>
-						</goals>
-					</execution>
-				</executions>
-				<configuration>
-					<jvmArgs>
-						<jvmArg>-Xms128m</jvmArg>
-						<jvmArg>-Xmx512m</jvmArg>
-					</jvmArgs>
-				</configuration>
-			</plugin>
-			
-			<!-- Eclipse Integration -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-eclipse-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<downloadSources>true</downloadSources>
-					<projectnatures>
-						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
-						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
-					</projectnatures>
-					<buildcommands>
-						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
-					</buildcommands>
-					<classpathContainers>
-						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
-						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
-					</classpathContainers>
-					<excludes>
-						<exclude>org.scala-lang:scala-library</exclude>
-						<exclude>org.scala-lang:scala-compiler</exclude>
-					</excludes>
-					<sourceIncludes>
-						<sourceInclude>**/*.scala</sourceInclude>
-						<sourceInclude>**/*.java</sourceInclude>
-					</sourceIncludes>
-				</configuration>
-			</plugin>
-
-			<!-- Adding scala source directories to build path -->
-			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>build-helper-maven-plugin</artifactId>
-				<version>1.7</version>
-				<executions>
-					<!-- Add src/main/scala to eclipse build path -->
-					<execution>
-						<id>add-source</id>
-						<phase>generate-sources</phase>
-						<goals>
-							<goal>add-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/main/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-					<!-- Add src/test/scala to eclipse build path -->
-					<execution>
-						<id>add-test-source</id>
-						<phase>generate-test-sources</phase>
-						<goals>
-							<goal>add-test-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/test/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<plugin>
-				<groupId>org.scalastyle</groupId>
-				<artifactId>scalastyle-maven-plugin</artifactId>
-				<version>0.5.0</version>
-				<executions>
-					<execution>
-						<goals>
-							<goal>check</goal>
-						</goals>
-					</execution>
-				</executions>
-				<configuration>
-					<verbose>false</verbose>
-					<failOnViolation>true</failOnViolation>
-					<includeTestSourceDirectory>true</includeTestSourceDirectory>
-					<failOnWarning>false</failOnWarning>
-					<sourceDirectory>${basedir}/src/main/scala</sourceDirectory>
-					<testSourceDirectory>${basedir}/src/test/scala</testSourceDirectory>
-					<configLocation>${project.basedir}/../../../tools/maven/scalastyle-config.xml</configLocation>
-					<outputFile>${project.basedir}/scalastyle-output.xml</outputFile>
-					<outputEncoding>UTF-8</outputEncoding>
-				</configuration>
-			</plugin>
-		</plugins>
-		
-		<pluginManagement>
-			<plugins>
-				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-				<plugin>
-					<groupId>org.eclipse.m2e</groupId>
-					<artifactId>lifecycle-mapping</artifactId>
-					<version>1.0.0</version>
-					<configuration>
-						<lifecycleMappingMetadata>
-							<pluginExecutions>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.maven.plugins</groupId>
-										<artifactId>maven-dependency-plugin</artifactId>
-										<versionRange>[2.9,)</versionRange>
-										<goals>
-											<goal>unpack</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-							</pluginExecutions>
-						</lifecycleMappingMetadata>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
deleted file mode 100644
index 2cf66b9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
+++ /dev/null
@@ -1,247 +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.examples.iteration;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.IterativeStream;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-/**
- * Example illustrating iterations in Flink streaming. <p/> <p> The program sums up random numbers and counts additions
- * it performs to reach a specific threshold in an iterative streaming fashion. </p>
- * <p/>
- * <p/>
- * This example shows how to use: <ul> <li>streaming iterations, <li>buffer timeout to enhance latency, <li>directed
- * outputs. </ul>
- */
-public class IterateExample {
-
-	private static final int BOUND = 100;
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up input for the stream of integer pairs
-
-		// obtain execution environment and set setBufferTimeout to 1 to enable
-		// continuous flushing of the output buffers (lowest latency)
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment()
-				.setBufferTimeout(1);
-
-		// create input stream of integer pairs
-		DataStream<Tuple2<Integer, Integer>> inputStream;
-		if (fileInput) {
-			inputStream = env.readTextFile(inputPath).map(new FibonacciInputMap());
-		} else {
-			inputStream = env.addSource(new RandomFibonacciSource());
-		}
-
-		// create an iterative data stream from the input with 5 second timeout
-		IterativeStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> it = inputStream.map(new InputMap())
-				.iterate(5000);
-
-		// apply the step function to get the next Fibonacci number
-		// increment the counter and split the output with the output selector
-		SplitStream<Tuple5<Integer, Integer, Integer, Integer, Integer>> step = it.map(new Step())
-				.split(new MySelector());
-
-		// close the iteration by selecting the tuples that were directed to the
-		// 'iterate' channel in the output selector
-		it.closeWith(step.select("iterate"));
-
-		// to produce the final output select the tuples directed to the
-		// 'output' channel then get the input pairs that have the greatest iteration counter
-		// on a 1 second sliding window
-		DataStream<Tuple2<Tuple2<Integer, Integer>, Integer>> numbers = step.select("output")
-				.map(new OutputMap());
-
-		// emit results
-		if (fileOutput) {
-			numbers.writeAsText(outputPath, 1);
-		} else {
-			numbers.print();
-		}
-
-		// execute the program
-		env.execute("Streaming Iteration Example");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Generate BOUND number of random integer pairs from the range from 0 to BOUND/2
-	 */
-	private static class RandomFibonacciSource implements SourceFunction<Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		private Random rnd = new Random();
-
-		private volatile boolean isRunning = true;
-		private int counter = 0;
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-
-			while (isRunning && counter < BOUND) {
-				int first = rnd.nextInt(BOUND / 2 - 1) + 1;
-				int second = rnd.nextInt(BOUND / 2 - 1) + 1;
-
-				ctx.collect(new Tuple2<Integer, Integer>(first, second));
-				counter++;
-				Thread.sleep(50L);
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-	}
-
-	/**
-	 * Generate random integer pairs from the range from 0 to BOUND/2
-	 */
-	private static class FibonacciInputMap implements MapFunction<String, Tuple2<Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Integer, Integer> map(String value) throws Exception {
-			String record = value.substring(1, value.length() - 1);
-			String[] splitted = record.split(",");
-			return new Tuple2<Integer, Integer>(Integer.parseInt(splitted[0]), Integer.parseInt(splitted[1]));
-		}
-	}
-
-	/**
-	 * Map the inputs so that the next Fibonacci numbers can be calculated while preserving the original input tuple A
-	 * counter is attached to the tuple and incremented in every iteration step
-	 */
-	public static class InputMap implements MapFunction<Tuple2<Integer, Integer>, Tuple5<Integer, Integer, Integer,
-			Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple5<Integer, Integer, Integer, Integer, Integer> map(Tuple2<Integer, Integer> value) throws
-				Exception {
-			return new Tuple5<Integer, Integer, Integer, Integer, Integer>(value.f0, value.f1, value.f0, value.f1, 0);
-		}
-	}
-
-	/**
-	 * Iteration step function that calculates the next Fibonacci number
-	 */
-	public static class Step implements
-			MapFunction<Tuple5<Integer, Integer, Integer, Integer, Integer>, Tuple5<Integer, Integer, Integer,
-					Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple5<Integer, Integer, Integer, Integer, Integer> map(Tuple5<Integer, Integer, Integer, Integer,
-				Integer> value) throws Exception {
-			return new Tuple5<Integer, Integer, Integer, Integer, Integer>(value.f0, value.f1, value.f3, value.f2 +
-					value.f3, ++value.f4);
-		}
-	}
-
-	/**
-	 * OutputSelector testing which tuple needs to be iterated again.
-	 */
-	public static class MySelector implements OutputSelector<Tuple5<Integer, Integer, Integer, Integer, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Iterable<String> select(Tuple5<Integer, Integer, Integer, Integer, Integer> value) {
-			List<String> output = new ArrayList<String>();
-			if (value.f2 < BOUND && value.f3 < BOUND) {
-				output.add("iterate");
-			} else {
-				output.add("output");
-			}
-			return output;
-		}
-	}
-
-	/**
-	 * Giving back the input pair and the counter
-	 */
-	public static class OutputMap implements MapFunction<Tuple5<Integer, Integer, Integer, Integer, Integer>,
-			Tuple2<Tuple2<Integer, Integer>, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Tuple2<Integer, Integer>, Integer> map(Tuple5<Integer, Integer, Integer, Integer, Integer>
-				value) throws
-				Exception {
-			return new Tuple2<Tuple2<Integer, Integer>, Integer>(new Tuple2<Integer, Integer>(value.f0, value.f1),
-					value.f4);
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileInput = false;
-	private static boolean fileOutput = false;
-	private static String inputPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			if (args.length == 1) {
-				fileOutput = true;
-				outputPath = args[0];
-			} else if (args.length == 2) {
-				fileInput = true;
-				inputPath = args[0];
-				fileOutput = true;
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: IterateExample <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing IterateExample with generated data.");
-			System.out.println("  Provide parameter to write to file.");
-			System.out.println("  Usage: IterateExample <result path>");
-		}
-		return true;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java
deleted file mode 100644
index 0077459..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/util/IterateExampleData.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.iteration.util;
-
-public class IterateExampleData {
-	public static final String INPUT_PAIRS = "(1,40)\n" + "(29,38)\n" + "(11,15)\n" + "(17,39)\n" + "(24,41)\n" +
-			"(7,33)\n" + "(20,2)\n" + "(11,5)\n" + "(3,16)\n" + "(23,36)\n" + "(15,23)\n" + "(28,13)\n" + "(1,1)\n" +
-			"(10,6)\n" + "(21,5)\n" + "(14,36)\n" + "(17,15)\n" + "(7,9)";
-
-	public static final String RESULTS = "((1,40),3)\n" + "((24,41),2)\n" + "((3,16),5)\n" + "((1,1),10)\n" +
-			"((17,15),4)\n" + "((29,38),2)\n" + "((7,33),3)\n" + "((23,36),3)\n" + "((10,6),6)\n" + "((7,9),5)\n" +
-			"((11,15),4)\n" + "((20,2),5)\n" + "((15,23),4)\n" + "((21,5),5)\n" +
-			"((17,39),3)\n" + "((11,5),6)\n" + "((28,13),4)\n" + "((14,36),3)";
-
-	private IterateExampleData() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
deleted file mode 100644
index 3355f1c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.join;
-
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Example illustrating join over sliding windows of streams in Flink.
- *
- * <p>
- * This example will join two streams with a sliding window. One which emits grades and one which
- * emits salaries of people. The input format for both sources has an additional timestamp
- * as field 0. This is used to to event-time windowing. Time timestamps must be
- * monotonically increasing.
- *
- * This example shows how to:
- * <ul>
- *   <li>do windowed joins,
- *   <li>use tuple data types,
- *   <li>write a simple streaming program.
- * </ul>
- */
-public class WindowJoin {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// obtain execution environment
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
-		// connect to the data sources for grades and salaries
-		Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> input = getInputStreams(env);
-		DataStream<Tuple3<Long, String, Integer>> grades = input.f0;
-		DataStream<Tuple3<Long, String, Integer>> salaries = input.f1;
-
-		// extract the timestamps
-		grades = grades.assignTimestamps(new MyTimestampExtractor());
-		salaries = salaries.assignTimestamps(new MyTimestampExtractor());
-
-		// apply a temporal join over the two stream based on the names over one
-		// second windows
-		DataStream<Tuple3<String, Integer, Integer>> joinedStream = grades
-				.join(salaries)
-				.where(new NameKeySelector())
-				.equalTo(new NameKeySelector())
-				.window(TumblingTimeWindows.of(Time.of(5, TimeUnit.MILLISECONDS)))
-				.apply(new MyJoinFunction());
-
-		// emit result
-		if (fileOutput) {
-			joinedStream.writeAsText(outputPath, 1);
-		} else {
-			joinedStream.print();
-		}
-
-		// execute program
-		env.execute("Windowed Join Example");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	private final static String[] names = {"tom", "jerry", "alice", "bob", "john", "grace"};
-	private final static int GRADE_COUNT = 5;
-	private final static int SALARY_MAX = 10000;
-	private final static int SLEEP_TIME = 10;
-
-	/**
-	 * Continuously emit tuples with random names and integers (grades).
-	 */
-	public static class GradeSource implements SourceFunction<Tuple3<Long, String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		private Random rand;
-		private Tuple3<Long, String, Integer> outTuple;
-		private volatile boolean isRunning = true;
-		private int counter;
-
-		public GradeSource() {
-			rand = new Random();
-			outTuple = new Tuple3<>();
-		}
-
-		@Override
-		public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
-			while (isRunning && counter < 100) {
-				outTuple.f0 = System.currentTimeMillis();
-				outTuple.f1 = names[rand.nextInt(names.length)];
-				outTuple.f2 = rand.nextInt(GRADE_COUNT) + 1;
-				Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
-				counter++;
-				ctx.collect(outTuple);
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-	}
-
-	/**
-	 * Continuously emit tuples with random names and integers (salaries).
-	 */
-	public static class SalarySource extends RichSourceFunction<Tuple3<Long, String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		private transient Random rand;
-		private transient Tuple3<Long, String, Integer> outTuple;
-		private volatile boolean isRunning;
-		private int counter;
-
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			rand = new Random();
-			outTuple = new Tuple3<Long, String, Integer>();
-			isRunning = true;
-		}
-
-
-		@Override
-		public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
-			while (isRunning && counter < 100) {
-				outTuple.f0 = System.currentTimeMillis();
-				outTuple.f1 = names[rand.nextInt(names.length)];
-				outTuple.f2 = rand.nextInt(SALARY_MAX) + 1;
-				Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
-				counter++;
-				ctx.collect(outTuple);
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-	}
-
-	public static class MySourceMap extends RichMapFunction<String, Tuple3<Long, String, Integer>> {
-
-		private static final long serialVersionUID = 1L;
-
-		private String[] record;
-
-		public MySourceMap() {
-			record = new String[2];
-		}
-
-		@Override
-		public Tuple3<Long, String, Integer> map(String line) throws Exception {
-			record = line.substring(1, line.length() - 1).split(",");
-			return new Tuple3<>(Long.parseLong(record[0]), record[1], Integer.parseInt(record[2]));
-		}
-	}
-
-	public static class MyJoinFunction
-			implements
-			JoinFunction<Tuple3<Long, String, Integer>, Tuple3<Long, String, Integer>, Tuple3<String, Integer, Integer>> {
-
-		private static final long serialVersionUID = 1L;
-
-		private Tuple3<String, Integer, Integer> joined = new Tuple3<>();
-
-		@Override
-		public Tuple3<String, Integer, Integer> join(Tuple3<Long, String, Integer> first,
-				Tuple3<Long, String, Integer> second) throws Exception {
-			joined.f0 = first.f1;
-			joined.f1 = first.f2;
-			joined.f2 = second.f2;
-			return joined;
-		}
-	}
-
-	private static class MyTimestampExtractor implements TimestampExtractor<Tuple3<Long, String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple3<Long, String, Integer> element, long currentTimestamp) {
-			return element.f0;
-		}
-
-		@Override
-		public long extractWatermark(Tuple3<Long, String, Integer> element, long currentTimestamp) {
-			return element.f0 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-
-	private static class NameKeySelector implements KeySelector<Tuple3<Long, String, Integer>, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String getKey(Tuple3<Long, String, Integer> value) throws Exception {
-			return value.f1;
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileInput = false;
-	private static boolean fileOutput = false;
-
-	private static String gradesPath;
-	private static String salariesPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			if (args.length == 1) {
-				fileOutput = true;
-				outputPath = args[0];
-			} else if (args.length == 3) {
-				fileInput = true;
-				fileOutput = true;
-				gradesPath = args[0];
-				salariesPath = args[1];
-				outputPath = args[2];
-			} else {
-				System.err.println("Usage: WindowJoin <result path> or WindowJoin <input path 1> <input path 2> " +
-						"<result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing WindowJoin with generated data.");
-			System.out.println("  Provide parameter to write to file.");
-			System.out.println("  Usage: WindowJoin <result path>");
-		}
-		return true;
-	}
-
-	private static Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> getInputStreams(
-			StreamExecutionEnvironment env) {
-
-		DataStream<Tuple3<Long, String, Integer>> grades;
-		DataStream<Tuple3<Long, String, Integer>> salaries;
-
-		if (fileInput) {
-			grades = env.readTextFile(gradesPath).map(new MySourceMap());
-			salaries = env.readTextFile(salariesPath).map(new MySourceMap());
-		} else {
-			grades = env.addSource(new GradeSource());
-			salaries = env.addSource(new SalarySource());
-		}
-
-		return Tuple2.of(grades, salaries);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
deleted file mode 100644
index 15c1280..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.join.util;
-
-public class WindowJoinData {
-
-	public static final String GRADES_INPUT = "(0,john,5)\n" + "(0,tom,3)\n" + "(0,alice,1)\n" + "(0,grace,5)\n" +
-			"(1,john,4)\n" + "(1,bob,1)\n" + "(1,alice,2)\n" + "(1,alice,3)\n" + "(1,bob,5)\n" + "(1,alice,3)\n" + "(1,tom,5)\n" +
-			"(2,john,2)\n" + "(2,john,1)\n" + "(2,grace,2)\n" + "(2,jerry,2)\n" + "(2,tom,4)\n" + "(2,bob,4)\n" + "(2,bob,2)\n" +
-			"(3, tom,2)\n" + "(3,alice,5)\n" + "(3,grace,5)\n" + "(3,grace,1)\n" + "(3,alice,1)\n" + "(3,grace,3)\n" + "(3,tom,1)\n" +
-			"(4,jerry,5)\n" + "(4,john,3)\n" + "(4,john,4)\n" + "(4,john,1)\n" + "(4,jerry,3)\n" + "(4,grace,3)\n" + "(4,bob,3)\n" +
-			"(5,john,3)\n" + "(5,jerry,4)\n" + "(5,tom,5)\n" + "(5,tom,4)\n" + "(5,john,2)\n" + "(5,jerry,1)\n" + "(5,bob,1)\n" +
-			"(6,john,5)\n" + "(6,grace,4)\n" + "(6,tom,5)\n" + "(6,john,4)\n" + "(6,tom,1)\n" + "(6,grace,1)\n" + "(6,john,2)\n" +
-			"(7,jerry,3)\n" + "(7,jerry,5)\n" + "(7,tom,2)\n" + "(7,tom,2)\n" + "(7,alice,4)\n" + "(7,tom,4)\n" + "(7,jerry,4)\n" +
-			"(8,john,3)\n" + "(8,grace,4)\n" + "(8,tom,3)\n" + "(8,jerry,4)\n" + "(8,john,5)\n" + "(8,john,4)\n" + "(8,jerry,1)\n" +
-			"(9,john,5)\n" + "(9,alice,2)\n" + "(9,tom,1)\n" + "(9,alice,5)\n" + "(9,grace,4)\n" + "(9,bob,4)\n" + "(9,jerry,1)\n" +
-			"(10,john,5)\n" + "(10,tom,4)\n" + "(10,tom,5)\n" + "(10,jerry,5)\n" + "(10,tom,1)\n" + "(10,grace,3)\n" + "(10,bob,5)\n" +
-			"(11,john,1)\n" + "(11,alice,1)\n" + "(11,grace,3)\n" + "(11,grace,1)\n" + "(11,jerry,1)\n" + "(11,jerry,4)\n" +
-			"(12,bob,4)\n" + "(12,alice,3)\n" + "(12,tom,5)\n" + "(12,alice,4)\n" + "(12,alice,4)\n" + "(12,grace,4)\n" + "(12,john,5)\n" +
-			"(13,john,5)\n" + "(13,grace,4)\n" + "(13,tom,4)\n" + "(13,john,4)\n" + "(13,john,5)\n" + "(13,alice,5)\n" + "(13,jerry,5)\n" +
-			"(14,john,3)\n" + "(14,tom,5)\n" + "(14,jerry,4)\n" + "(14,grace,4)\n" + "(14,john,3)\n" + "(14,bob,2)";
-
-	public static final String SALARIES_INPUT = "(0,john,6469)\n" + "(0,jerry,6760)\n" + "(0,jerry,8069)\n" +
-			"(1,tom,3662)\n" + "(1,grace,8427)\n" + "(1,john,9425)\n" + "(1,bob,9018)\n" + "(1,john,352)\n" + "(1,tom,3770)\n" +
-			"(2,grace,7622)\n" + "(2,jerry,7441)\n" + "(2,alice,1468)\n" + "(2,bob,5472)\n" + "(2,grace,898)\n" +
-			"(3,tom,3849)\n" + "(3,grace,1865)\n" + "(3,alice,5582)\n" + "(3,john,9511)\n" + "(3,alice,1541)\n" +
-			"(4,john,2477)\n" + "(4,grace,3561)\n" + "(4,john,1670)\n" + "(4,grace,7290)\n" + "(4,grace,6565)\n" +
-			"(5,tom,6179)\n" + "(5,tom,1601)\n" + "(5,john,2940)\n" + "(5,bob,4685)\n" + "(5,bob,710)\n" + "(5,bob,5936)\n" +
-			"(6,jerry,1412)\n" + "(6,grace,6515)\n" + "(6,grace,3321)\n" + "(6,tom,8088)\n" + "(6,john,2876)\n" +
-			"(7,bob,9896)\n" + "(7,grace,7368)\n" + "(7,grace,9749)\n" + "(7,bob,2048)\n" + "(7,alice,4782)\n" +
-			"(8,alice,3375)\n" + "(8,tom,5841)\n" + "(8,bob,958)\n" + "(8,bob,5258)\n" + "(8,tom,3935)\n" + "(8,jerry,4394)\n" +
-			"(9,alice,102)\n" + "(9,alice,4931)\n" + "(9,alice,5240)\n" + "(9,jerry,7951)\n" + "(9,john,5675)\n" +
-			"(10,bob,609)\n" + "(10,alice,5997)\n" + "(10,jerry,9651)\n" + "(10,alice,1328)\n" + "(10,bob,1022)\n" +
-			"(11,grace,2578)\n" + "(11,jerry,9704)\n" + "(11,tom,4476)\n" + "(11,grace,3784)\n" + "(11,alice,6144)\n" +
-			"(12,bob,6213)\n" + "(12,alice,7525)\n" + "(12,jerry,2908)\n" + "(12,grace,8464)\n" + "(12,jerry,9920)\n" +
-			"(13,bob,3720)\n" + "(13,bob,7612)\n" + "(13,alice,7211)\n" + "(13,jerry,6484)\n" + "(13,alice,1711)\n" +
-			"(14,jerry,5994)\n" + "(14,grace,928)\n" + "(14,jerry,2492)\n" + "(14,grace,9080)\n" + "(14,tom,4330)\n" +
-			"(15,bob,8302)\n" + "(15,john,4981)\n" + "(15,tom,1781)\n" + "(15,grace,1379)\n" + "(15,jerry,3700)\n" +
-			"(16,jerry,3584)\n" + "(16,jerry,2038)\n" + "(16,jerry,3902)\n" + "(16,tom,1336)\n" + "(16,jerry,7500)\n" +
-			"(17,tom,3648)\n" + "(17,alice,2533)\n" + "(17,tom,8685)\n" + "(17,bob,3968)\n" + "(17,tom,3241)\n" + "(17,bob,7461)\n" +
-			"(18,jerry,2138)\n" + "(18,alice,7503)\n" + "(18,alice,6424)\n" + "(18,tom,140)\n" + "(18,john,9802)\n" +
-			"(19,grace,2977)\n" + "(19,grace,889)\n" + "(19,john,1338)";
-
-	private WindowJoinData() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
deleted file mode 100644
index ce227e4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.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.examples.ml;
-
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.util.Collector;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Skeleton for incremental machine learning algorithm consisting of a
- * pre-computed model, which gets updated for the new inputs and new input data
- * for which the job provides predictions.
- *
- * <p>
- * This may serve as a base of a number of algorithms, e.g. updating an
- * incremental Alternating Least Squares model while also providing the
- * predictions.
- *
- * <p/>
- * This example shows how to use:
- * <ul>
- *   <li>Connected streams
- *   <li>CoFunctions
- *   <li>Tuple data types
- * </ul>
- */
-public class IncrementalLearningSkeleton {
-
-	private static DataStream<Integer> trainingData = null;
-	private static DataStream<Integer> newData = null;
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
-		trainingData = env.addSource(new FiniteTrainingDataSource());
-		newData = env.addSource(new FiniteNewDataSource());
-
-		// build new model on every second of new data
-		DataStream<Double[]> model = trainingData
-				.assignTimestamps(new LinearTimestamp())
-				.timeWindowAll(Time.of(5000, TimeUnit.MILLISECONDS))
-				.apply(new PartialModelBuilder());
-
-		// use partial model for newData
-		DataStream<Integer> prediction = newData.connect(model).map(new Predictor());
-
-		// emit result
-		if (fileOutput) {
-			prediction.writeAsText(outputPath, 1);
-		} else {
-			prediction.print();
-		}
-
-		// execute program
-		env.execute("Streaming Incremental Learning");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Feeds new data for newData. By default it is implemented as constantly
-	 * emitting the Integer 1 in a loop.
-	 */
-	public static class FiniteNewDataSource implements SourceFunction<Integer> {
-		private static final long serialVersionUID = 1L;
-		private int counter;
-
-		@Override
-		public void run(SourceContext<Integer> ctx) throws Exception {
-			Thread.sleep(15);
-			while (counter < 50) {
-				ctx.collect(getNewData());
-			}
-		}
-
-		@Override
-		public void cancel() {
-			// No cleanup needed
-		}
-
-		private Integer getNewData() throws InterruptedException {
-			Thread.sleep(5);
-			counter++;
-			return 1;
-		}
-	}
-
-	/**
-	 * Feeds new training data for the partial model builder. By default it is
-	 * implemented as constantly emitting the Integer 1 in a loop.
-	 */
-	public static class FiniteTrainingDataSource implements SourceFunction<Integer> {
-		private static final long serialVersionUID = 1L;
-		private int counter = 0;
-
-		@Override
-		public void run(SourceContext<Integer> collector) throws Exception {
-			while (counter < 8200) {
-				collector.collect(getTrainingData());
-			}
-		}
-
-		@Override
-		public void cancel() {
-			// No cleanup needed
-		}
-
-		private Integer getTrainingData() throws InterruptedException {
-			counter++;
-			return 1;
-		}
-	}
-
-	public static class LinearTimestamp implements TimestampExtractor<Integer> {
-		private static final long serialVersionUID = 1L;
-
-		private long counter = 0L;
-
-		@Override
-		public long extractTimestamp(Integer element, long currentTimestamp) {
-			return counter += 10L;
-		}
-
-		@Override
-		public long extractWatermark(Integer element, long currentTimestamp) {
-			return counter - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-
-	}
-
-	/**
-	 * Builds up-to-date partial models on new training data.
-	 */
-	public static class PartialModelBuilder implements AllWindowFunction<Integer, Double[], TimeWindow> {
-		private static final long serialVersionUID = 1L;
-
-		protected Double[] buildPartialModel(Iterable<Integer> values) {
-			return new Double[]{1.};
-		}
-
-		@Override
-		public void apply(TimeWindow window, Iterable<Integer> values, Collector<Double[]> out) throws Exception {
-			out.collect(buildPartialModel(values));
-		}
-	}
-
-	/**
-	 * Creates newData using the model produced in batch-processing and the
-	 * up-to-date partial model.
-	 * <p/>
-	 * <p>
-	 * By defaults emits the Integer 0 for every newData and the Integer 1
-	 * for every model update.
-	 * </p>
-	 */
-	public static class Predictor implements CoMapFunction<Integer, Double[], Integer> {
-		private static final long serialVersionUID = 1L;
-
-		Double[] batchModel = null;
-		Double[] partialModel = null;
-
-		@Override
-		public Integer map1(Integer value) {
-			// Return newData
-			return predict(value);
-		}
-
-		@Override
-		public Integer map2(Double[] value) {
-			// Update model
-			partialModel = value;
-			batchModel = getBatchModel();
-			return 1;
-		}
-
-		// pulls model built with batch-job on the old training data
-		protected Double[] getBatchModel() {
-			return new Double[]{0.};
-		}
-
-		// performs newData using the two models
-		protected Integer predict(Integer inTuple) {
-			return 0;
-		}
-
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 1) {
-				outputPath = args[0];
-			} else {
-				System.err.println("Usage: IncrementalLearningSkeleton <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing IncrementalLearningSkeleton with generated data.");
-			System.out.println("  Provide parameter to write to file.");
-			System.out.println("  Usage: IncrementalLearningSkeleton <result path>");
-		}
-		return true;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
deleted file mode 100644
index 8a6cd88..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.ml.util;
-
-public class IncrementalLearningSkeletonData {
-
-	public static final String RESULTS = "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" +
-			"1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "0\n" + "0\n" +
-			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
-			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
-			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
-			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
-			"0\n" + "0\n" + "0\n" + "0\n";
-
-	private IncrementalLearningSkeletonData() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
deleted file mode 100644
index 17add2c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.socket;
-
-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.examples.wordcount.WordCount.Tokenizer;
-
-/**
- * This example shows an implementation of WordCount with data from a text
- * socket. To run the example make sure that the service providing the text data
- * is already up and running.
- * <p/>
- * <p/>
- * To start an example socket text stream on your local machine run netcat from
- * a command line: <code>nc -lk 9999</code>, where the parameter specifies the
- * port number.
- * <p/>
- * <p/>
- * <p/>
- * Usage:
- * <code>SocketTextStreamWordCount &lt;hostname&gt; &lt;port&gt; &lt;result path&gt;</code>
- * <br>
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>use StreamExecutionEnvironment.socketTextStream
- * <li>write a simple Flink program,
- * <li>write and use user-defined functions.
- * </ul>
- *
- * @see <a href="www.openbsd.org/cgi-bin/man.cgi?query=nc">netcat</a>
- */
-public class SocketTextStreamWordCount {
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment
-				.getExecutionEnvironment();
-
-		// get input data
-		DataStream<String> text = env.socketTextStream(hostName, port, '\n', 0);
-
-		DataStream<Tuple2<String, Integer>> counts =
-				// split up the lines in pairs (2-tuples) containing: (word,1)
-				text.flatMap(new Tokenizer())
-						// group by the tuple field "0" and sum up tuple field "1"
-						.keyBy(0)
-						.sum(1);
-
-		if (fileOutput) {
-			counts.writeAsText(outputPath, 1);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("WordCount from SocketTextStream Example");
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String hostName;
-	private static int port;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		// parse input arguments
-		if (args.length == 3) {
-			fileOutput = true;
-			hostName = args[0];
-			port = Integer.valueOf(args[1]);
-			outputPath = args[2];
-		} else if (args.length == 2) {
-			hostName = args[0];
-			port = Integer.valueOf(args[1]);
-		} else {
-			System.err.println("Usage: SocketTextStreamWordCount <hostname> <port> [<output path>]");
-			return false;
-		}
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
deleted file mode 100644
index c2477b5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.twitter;
-
-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.connectors.json.JSONParseFlatMap;
-import org.apache.flink.streaming.connectors.twitter.TwitterSource;
-import org.apache.flink.streaming.examples.twitter.util.TwitterStreamData;
-import org.apache.flink.util.Collector;
-import org.apache.sling.commons.json.JSONException;
-
-import java.util.StringTokenizer;
-
-/**
- * Implements the "TwitterStream" program that computes a most used word
- * occurrence over JSON files in a streaming fashion.
- * <p/>
- * <p/>
- * The input is a JSON text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>TwitterStream &lt;text path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from
- * {@link TwitterStreamData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>acquire external data,
- * <li>use in-line defined functions,
- * <li>handle flattened stream inputs.
- * </ul>
- */
-public class TwitterStream {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		DataStream<String> streamSource = getTextDataStream(env);
-
-		DataStream<Tuple2<String, Integer>> tweets = streamSource
-				// selecting English tweets and splitting to (word, 1)
-				.flatMap(new SelectEnglishAndTokenizeFlatMap())
-				// group by words and sum their occurrences
-				.keyBy(0).sum(1);
-
-		// emit result
-		if (fileOutput) {
-			tweets.writeAsText(outputPath);
-		} else {
-			tweets.print();
-		}
-
-		// execute program
-		env.execute("Twitter Streaming Example");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Makes sentences from English tweets.
-	 * <p/>
-	 * <p>
-	 * Implements a string tokenizer that splits sentences into words as a
-	 * user-defined FlatMapFunction. The function takes a line (String) and
-	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
-	 * Integer>).
-	 * </p>
-	 */
-	public static class SelectEnglishAndTokenizeFlatMap extends JSONParseFlatMap<String, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		/**
-		 * Select the language from the incoming JSON text
-		 */
-		@Override
-		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception {
-			try {
-				if (getString(value, "user.lang").equals("en")) {
-					// message of tweet
-					StringTokenizer tokenizer = new StringTokenizer(getString(value, "text"));
-
-					// split the message
-					while (tokenizer.hasMoreTokens()) {
-						String result = tokenizer.nextToken().replaceAll("\\s*", "").toLowerCase();
-
-						if (result != null && !result.equals("")) {
-							out.collect(new Tuple2<String, Integer>(result, 1));
-						}
-					}
-				}
-			} catch (JSONException e) {
-				// the JSON was not parsed correctly
-			}
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileInput = false;
-	private static boolean fileOutput = false;
-	private static String propertiesPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				fileInput = true;
-				propertiesPath = args[0];
-				outputPath = args[1];
-			} else if (args.length == 1) {
-				outputPath = args[0];
-			} else {
-				System.err.println("USAGE:\nTwitterStream [<pathToPropertiesFile>] <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing TwitterStream example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from a file.");
-			System.out.println("  USAGE: TwitterStream [<pathToPropertiesFile>] <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
-		if (fileInput) {
-			// read the text file from given input path
-			return env.addSource(new TwitterSource(propertiesPath));
-		} else {
-			// get default test text data
-			return env.fromElements(TwitterStreamData.TEXTS);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java
deleted file mode 100644
index b06d193..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.twitter.util;
-
-//example data looking like tweets, but not acquired from Twitter
-public class TwitterStreamData {
-	public static final String[] TEXTS = new String[] {
-			"{\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"id\":000000000000000000,\"id_str\":\"000000000000000000\",\"text\":\"Apache Flink\",\"source\":null,\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":0000000000,\"id_str\":\"0000000000\",\"name\":\"Apache Flink\",\"screen_name\":\"Apache Flink\",\"location\":\"Berlin\",\"protected\":false,\"verified\":false,\"followers_count\":999999,\"friends_count\":99999,\"listed_count\":999,\"favourites_count\":9999,\"statuses_count\":999,\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"utc_offset\":7200,\"time_zone\":\"Amsterdam\",\"geo_enabled\":false,\"lang\":\"en\",\"entities\":{\"hashtags\":[{\"text\":\"example1\",\"indices\":[0,0]},{\"text\":\"tweet1\",\"indices\":[0,0]}]},\"contributors_enabled\":false,\"is_translator\":false,\"profile_background_color\":\"C6E2EE\",\"profile_background_
 tile\":false,\"profile_link_color\":\"1F98C7\",\"profile_sidebar_border_color\":\"FFFFFF\",\"profile_sidebar_fill_color\":\"252429\",\"profile_text_color\":\"666666\",\"profile_use_background_image\":true,\"default_profile\":false,\"default_profile_image\":false,\"following\":null,\"follow_request_sent\":null,\"notifications\":null},\"geo\":null,\"coordinates\":null,\"place\":null,\"contributors\":null}",
-			"{\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"id\":000000000000000001,\"id_str\":\"000000000000000000\",\"text\":\"Apache Flink\",\"source\":null,\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":0000000000,\"id_str\":\"0000000000\",\"name\":\"Apache Flink\",\"screen_name\":\"Apache Flink\",\"location\":\"Berlin\",\"protected\":false,\"verified\":false,\"followers_count\":999999,\"friends_count\":99999,\"listed_count\":999,\"favourites_count\":9999,\"statuses_count\":999,\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"utc_offset\":7200,\"time_zone\":\"Amsterdam\",\"geo_enabled\":false,\"lang\":\"en\",\"entities\":{\"hashtags\":[{\"text\":\"example2\",\"indices\":[0,0]},{\"text\":\"tweet2\",\"indices\":[0,0]}]},\"contributors_enabled\":false,\"is_translator\":false,\"profile_background_color\":\"C6E2EE\",\"profile_background_
 tile\":false,\"profile_link_color\":\"1F98C7\",\"profile_sidebar_border_color\":\"FFFFFF\",\"profile_sidebar_fill_color\":\"252429\",\"profile_text_color\":\"666666\",\"profile_use_background_image\":true,\"default_profile\":false,\"default_profile_image\":false,\"following\":null,\"follow_request_sent\":null,\"notifications\":null},\"geo\":null,\"coordinates\":null,\"place\":null,\"contributors\":null}",
-			"{\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"id\":000000000000000002,\"id_str\":\"000000000000000000\",\"text\":\"Apache Flink\",\"source\":null,\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":0000000000,\"id_str\":\"0000000000\",\"name\":\"Apache Flink\",\"screen_name\":\"Apache Flink\",\"location\":\"Berlin\",\"protected\":false,\"verified\":false,\"followers_count\":999999,\"friends_count\":99999,\"listed_count\":999,\"favourites_count\":9999,\"statuses_count\":999,\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"utc_offset\":7200,\"time_zone\":\"Amsterdam\",\"geo_enabled\":false,\"lang\":\"en\",\"entities\":{\"hashtags\":[{\"text\":\"example3\",\"indices\":[0,0]},{\"text\":\"tweet3\",\"indices\":[0,0]}]},\"contributors_enabled\":false,\"is_translator\":false,\"profile_background_color\":\"C6E2EE\",\"profile_background_
 tile\":false,\"profile_link_color\":\"1F98C7\",\"profile_sidebar_border_color\":\"FFFFFF\",\"profile_sidebar_fill_color\":\"252429\",\"profile_text_color\":\"666666\",\"profile_use_background_image\":true,\"default_profile\":false,\"default_profile_image\":false,\"following\":null,\"follow_request_sent\":null,\"notifications\":null},\"geo\":null,\"coordinates\":null,\"place\":null,\"contributors\":null}",
-	};
-
-	public static final String STREAMING_COUNTS_AS_TUPLES = "(apache,1)\n" + "(apache,2)\n" + "(apache,3)\n" + "(flink,1)\n" + "(flink,2)\n" + "(flink,3)\n";
-
-	private TwitterStreamData() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 982b73d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
+++ /dev/null
@@ -1,127 +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.examples.windowing;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-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.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
-
-@SuppressWarnings("serial")
-public class GroupedProcessingTimeWindowExample {
-	
-	public static void main(String[] args) throws Exception {
-		
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(4);
-		
-		DataStream<Tuple2<Long, Long>> stream = env
-				.addSource(new RichParallelSourceFunction<Tuple2<Long, Long>>() {
-					
-					private volatile boolean running = true;
-					
-					@Override
-					public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
-						
-						final long startTime = System.currentTimeMillis();
-						
-						final long numElements = 20000000;
-						final long numKeys = 10000;
-						long val = 1L;
-						long count = 0L;
-						
-						
-						while (running && count < numElements) {
-							count++;
-							ctx.collect(new Tuple2<Long, Long>(val++, 1L));
-							
-							if (val > numKeys) {
-								val = 1L;
-							}
-						}
-
-						final long endTime = System.currentTimeMillis();
-						System.out.println("Took " + (endTime-startTime) + " msecs for " + numElements + " values");
-					}
-
-					@Override
-					public void cancel() {
-						running = false;
-					}
-				});
-		
-		stream
-			.keyBy(0)
-			.timeWindow(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
-			.reduce(new SummingReducer())
-
-			// alternative: use a apply function which does not pre-aggregate
-//			.keyBy(new FirstFieldKeyExtractor<Tuple2<Long, Long>, Long>())
-//			.window(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
-//			.apply(new SummingWindowFunction())
-				
-			.addSink(new SinkFunction<Tuple2<Long, Long>>() {
-				@Override
-				public void invoke(Tuple2<Long, Long> value) {
-				}
-			});
-		
-		env.execute();
-	}
-	
-	public static class FirstFieldKeyExtractor<Type extends Tuple, Key> implements KeySelector<Type, Key> {
-		
-		@Override
-		@SuppressWarnings("unchecked")
-		public Key getKey(Type value) {
-			return (Key) value.getField(0);
-		}
-	}
-
-	public static class SummingWindowFunction implements WindowFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Long, Window> {
-
-		@Override
-		public void apply(Long key, Window window, Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) {
-			long sum = 0L;
-			for (Tuple2<Long, Long> value : values) {
-				sum += value.f1;
-			}
-
-			out.collect(new Tuple2<>(key, sum));
-		}
-	}
-
-	public static class SummingReducer implements ReduceFunction<Tuple2<Long, Long>> {
-
-		@Override
-		public Tuple2<Long, Long> reduce(Tuple2<Long, Long> value1, Tuple2<Long, Long> value2) {
-			return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
deleted file mode 100644
index 3c63156..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
+++ /dev/null
@@ -1,168 +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.examples.windowing;
-
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
-import org.apache.flink.streaming.api.windowing.triggers.Trigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class SessionWindowing {
-
-	@SuppressWarnings("serial")
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(2);
-
-		final List<Tuple3<String, Long, Integer>> input = new ArrayList<>();
-
-		input.add(new Tuple3<>("a", 1L, 1));
-		input.add(new Tuple3<>("b", 1L, 1));
-		input.add(new Tuple3<>("b", 3L, 1));
-		input.add(new Tuple3<>("b", 5L, 1));
-		input.add(new Tuple3<>("c", 6L, 1));
-		// We expect to detect the session "a" earlier than this point (the old
-		// functionality can only detect here when the next starts)
-		input.add(new Tuple3<>("a", 10L, 1));
-		// We expect to detect session "b" and "c" at this point as well
-		input.add(new Tuple3<>("c", 11L, 1));
-
-		DataStream<Tuple3<String, Long, Integer>> source = env
-				.addSource(new EventTimeSourceFunction<Tuple3<String,Long,Integer>>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void run(SourceContext<Tuple3<String, Long, Integer>> ctx) throws Exception {
-						for (Tuple3<String, Long, Integer> value : input) {
-							ctx.collectWithTimestamp(value, value.f1);
-							ctx.emitWatermark(new Watermark(value.f1 - 1));
-							if (!fileOutput) {
-								System.out.println("Collected: " + value);
-							}
-						}
-						ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
-					}
-
-					@Override
-					public void cancel() {
-					}
-				});
-
-		// We create sessions for each id with max timeout of 3 time units
-		DataStream<Tuple3<String, Long, Integer>> aggregated = source
-				.keyBy(0)
-				.window(GlobalWindows.create())
-				.trigger(new SessionTrigger(3L))
-				.sum(2);
-
-		if (fileOutput) {
-			aggregated.writeAsText(outputPath);
-		} else {
-			aggregated.print();
-		}
-
-		env.execute();
-	}
-
-	private static class SessionTrigger implements Trigger<Tuple3<String, Long, Integer>, GlobalWindow> {
-
-		private static final long serialVersionUID = 1L;
-
-		private final Long sessionTimeout;
-
-		public SessionTrigger(Long sessionTimeout) {
-			this.sessionTimeout = sessionTimeout;
-
-		}
-
-		@Override
-		public TriggerResult onElement(Tuple3<String, Long, Integer> element, long timestamp, GlobalWindow window, TriggerContext ctx) throws Exception {
-
-			OperatorState<Long> lastSeenState = ctx.getKeyValueState("last-seen", 1L);
-			Long lastSeen = lastSeenState.value();
-
-			Long timeSinceLastEvent = timestamp - lastSeen;
-
-			// Update the last seen event time
-			lastSeenState.update(timestamp);
-
-			ctx.registerEventTimeTimer(lastSeen + sessionTimeout);
-
-			if (timeSinceLastEvent > sessionTimeout) {
-				return TriggerResult.FIRE_AND_PURGE;
-			} else {
-				return TriggerResult.CONTINUE;
-			}
-		}
-
-		@Override
-		public TriggerResult onEventTime(long time, TriggerContext ctx) throws Exception {
-			OperatorState<Long> lastSeenState = ctx.getKeyValueState("last-seen", 1L);
-			Long lastSeen = lastSeenState.value();
-
-			if (time - lastSeen >= sessionTimeout) {
-				return TriggerResult.FIRE_AND_PURGE;
-			}
-			return TriggerResult.CONTINUE;
-		}
-
-		@Override
-		public TriggerResult onProcessingTime(long time,
-				TriggerContext ctx) throws Exception {
-			return TriggerResult.CONTINUE;
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			if (args.length == 1) {
-				fileOutput = true;
-				outputPath = args[0];
-			} else {
-				System.err.println("Usage: SessionWindowing <result path>");
-				return false;
-			}
-		}
-		return true;
-	}
-
-}


[13/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
deleted file mode 100644
index 988e7ec..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ /dev/null
@@ -1,543 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import java.lang
-import org.apache.flink.api.common.functions._
-import org.apache.flink.api.java.typeutils.TypeExtractor
-import org.apache.flink.streaming.api.collector.selector.OutputSelector
-import org.apache.flink.streaming.api.functions.co.CoMapFunction
-import org.apache.flink.streaming.api.graph.{StreamEdge, StreamGraph}
-import org.apache.flink.streaming.api.operators.{AbstractUdfStreamOperator, StreamOperator}
-import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
-import org.apache.flink.streaming.api.windowing.triggers.{PurgingTrigger, CountTrigger}
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow
-import org.apache.flink.streaming.api.scala._
-import org.apache.flink.streaming.runtime.partitioner._
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.apache.flink.util.Collector
-
-import org.junit.Assert.fail
-import org.junit.Test
-
-class DataStreamTest extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testNaming(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source1Operator = env.generateSequence(0, 0).name("testSource1")
-    val source1 = source1Operator
-    assert("testSource1" == source1Operator.getName)
-
-    val dataStream1 = source1
-      .map(x => 0L)
-      .name("testMap")
-    assert("testMap" == dataStream1.getName)
-
-    val dataStream2 = env.generateSequence(0, 0).name("testSource2")
-      .keyBy(x=>x)
-      .reduce((x, y) => 0)
-      .name("testReduce")
-    assert("testReduce" == dataStream2.getName)
-
-    val connected = dataStream1.connect(dataStream2)
-      .flatMap({ (in, out: Collector[(Long, Long)]) => }, { (in, out: Collector[(Long, Long)]) => })
-      .name("testCoFlatMap")
-
-    assert("testCoFlatMap" == connected.getName)
-
-    val func: (((Long, Long), (Long, Long)) => (Long, Long)) =
-      (x: (Long, Long), y: (Long, Long)) => (0L, 0L)
-
-    val windowed = connected
-      .windowAll(GlobalWindows.create())
-      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](10)))
-      .fold((0L, 0L), func)
-
-    windowed.name("testWindowFold")
-
-    assert("testWindowFold" == windowed.getName)
-
-    windowed.print()
-
-    val plan = env.getExecutionPlan
-
-    assert(plan contains "testSource1")
-    assert(plan contains "testSource2")
-    assert(plan contains "testMap")
-    assert(plan contains "testReduce")
-    assert(plan contains "testCoFlatMap")
-    assert(plan contains "testWindowFold")
-  }
-
-  /**
-   * Tests that {@link DataStream#keyBy} and {@link DataStream#partitionBy(KeySelector)} result in
-   * different and correct topologies. Does the some for the {@link ConnectedStreams}.
-   */
-  @Test
-  def testPartitioning(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val src1: DataStream[(Long, Long)] = env.fromElements((0L, 0L))
-    val src2: DataStream[(Long, Long)] = env.fromElements((0L, 0L))
-
-    val connected = src1.connect(src2)
-
-    val group1 = src1.keyBy(0)
-    val group2 = src1.keyBy(1, 0)
-    val group3 = src1.keyBy("_1")
-    val group4 = src1.keyBy(x => x._1)
-
-    val gid1 = createDownStreamId(group1)
-    val gid2 = createDownStreamId(group2)
-    val gid3 = createDownStreamId(group3)
-    val gid4 = createDownStreamId(group4)
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid1)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid2)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid3)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, gid4)))
-
-    //Testing DataStream partitioning
-    val partition1: DataStream[_] = src1.partitionByHash(0)
-    val partition2: DataStream[_] = src1.partitionByHash(1, 0)
-    val partition3: DataStream[_] = src1.partitionByHash("_1")
-    val partition4: DataStream[_] = src1.partitionByHash((x : (Long, Long)) => x._1)
-
-    val pid1 = createDownStreamId(partition1)
-    val pid2 = createDownStreamId(partition2)
-    val pid3 = createDownStreamId(partition3)
-    val pid4 = createDownStreamId(partition4)
-
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid1)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid2)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid3)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, pid4)))
-
-    // Testing DataStream custom partitioning
-    val longPartitioner: Partitioner[Long] = new Partitioner[Long] {
-      override def partition(key: Long, numPartitions: Int): Int = 0
-    }
-
-    val customPartition1: DataStream[_] =
-      src1.partitionCustom(longPartitioner, 0)
-    val customPartition3: DataStream[_] =
-      src1.partitionCustom(longPartitioner, "_1")
-    val customPartition4: DataStream[_] =
-      src1.partitionCustom(longPartitioner, (x : (Long, Long)) => x._1)
-
-    val cpid1 = createDownStreamId(customPartition1)
-    val cpid2 = createDownStreamId(customPartition3)
-    val cpid3 = createDownStreamId(customPartition4)
-    assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid1)))
-    assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid2)))
-    assert(isCustomPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, cpid3)))
-
-    //Testing ConnectedStreams grouping
-    val connectedGroup1: ConnectedStreams[_, _] = connected.keyBy(0, 0)
-    val downStreamId1: Integer = createDownStreamId(connectedGroup1)
-
-    val connectedGroup2: ConnectedStreams[_, _] = connected.keyBy(Array[Int](0), Array[Int](0))
-    val downStreamId2: Integer = createDownStreamId(connectedGroup2)
-
-    val connectedGroup3: ConnectedStreams[_, _] = connected.keyBy("_1", "_1")
-    val downStreamId3: Integer = createDownStreamId(connectedGroup3)
-
-    val connectedGroup4: ConnectedStreams[_, _] =
-      connected.keyBy(Array[String]("_1"), Array[String]("_1"))
-    val downStreamId4: Integer = createDownStreamId(connectedGroup4)
-
-    val connectedGroup5: ConnectedStreams[_, _] = connected.keyBy(x => x._1, x => x._1)
-    val downStreamId5: Integer = createDownStreamId(connectedGroup5)
-
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId1)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId1)))
-
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId2)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId2)))
-
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId3)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId3)))
-
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId4)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId4)))
-
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, downStreamId5)))
-    assert(isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, downStreamId5)))
-
-    //Testing ConnectedStreams partitioning
-    val connectedPartition1: ConnectedStreams[_, _] = connected.partitionByHash(0, 0)
-    val connectDownStreamId1: Integer = createDownStreamId(connectedPartition1)
-
-    val connectedPartition2: ConnectedStreams[_, _] =
-      connected.partitionByHash(Array[Int](0), Array[Int](0))
-    val connectDownStreamId2: Integer = createDownStreamId(connectedPartition2)
-
-    val connectedPartition3: ConnectedStreams[_, _] = connected.partitionByHash("_1", "_1")
-    val connectDownStreamId3: Integer = createDownStreamId(connectedPartition3)
-
-    val connectedPartition4: ConnectedStreams[_, _] =
-      connected.partitionByHash(Array[String]("_1"), Array[String]("_1"))
-    val connectDownStreamId4: Integer = createDownStreamId(connectedPartition4)
-
-    val connectedPartition5: ConnectedStreams[_, _] =
-      connected.partitionByHash(x => x._1, x => x._1)
-    val connectDownStreamId5: Integer = createDownStreamId(connectedPartition5)
-
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId1))
-    )
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId1))
-    )
-
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId2))
-    )
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId2))
-    )
-
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId3))
-    )
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId3))
-    )
-
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId4))
-    )
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId4))
-    )
-
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src1.getId, connectDownStreamId5))
-    )
-    assert(
-      isPartitioned(env.getStreamGraph.getStreamEdge(src2.getId, connectDownStreamId5))
-    )
-  }
-
-  /**
-   * Tests whether parallelism gets set.
-   */
-  @Test
-  def testParallelism() {
-    val env: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment(10)
-
-    val src = env.fromElements(new Tuple2[Long, Long](0L, 0L))
-    val map = src.map(x => (0L, 0L))
-    val windowed: DataStream[(Long, Long)] = map
-      .windowAll(GlobalWindows.create())
-      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](10)))
-      .fold((0L, 0L), (x: (Long, Long), y: (Long, Long)) => (0L, 0L))
-
-    windowed.print()
-    val sink = map.addSink(x => {})
-
-    assert(1 == env.getStreamGraph.getStreamNode(src.getId).getParallelism)
-    assert(10 == env.getStreamGraph.getStreamNode(map.getId).getParallelism)
-    assert(1 == env.getStreamGraph.getStreamNode(windowed.getId).getParallelism)
-    assert(10 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism)
-
-    try {
-      src.setParallelism(3)
-      fail()
-    }
-    catch {
-      case success: IllegalArgumentException => {
-      }
-    }
-
-    env.setParallelism(7)
-    // the parallelism does not change since some windowing code takes the parallelism from
-    // input operations and that cannot change dynamically
-    assert(1 == env.getStreamGraph.getStreamNode(src.getId).getParallelism)
-    assert(10 == env.getStreamGraph.getStreamNode(map.getId).getParallelism)
-    assert(1 == env.getStreamGraph.getStreamNode(windowed.getId).getParallelism)
-    assert(10 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism)
-
-    val parallelSource = env.generateSequence(0, 0)
-    parallelSource.print()
-
-    assert(7 == env.getStreamGraph.getStreamNode(parallelSource.getId).getParallelism)
-
-    parallelSource.setParallelism(3)
-    assert(3 == env.getStreamGraph.getStreamNode(parallelSource.getId).getParallelism)
-
-    map.setParallelism(2)
-    assert(2 == env.getStreamGraph.getStreamNode(map.getId).getParallelism)
-
-    sink.setParallelism(4)
-    assert(4 == env.getStreamGraph.getStreamNode(sink.getTransformation.getId).getParallelism)
-  }
-
-  @Test
-  def testTypeInfo() {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val src1: DataStream[Long] = env.generateSequence(0, 0)
-    assert(TypeExtractor.getForClass(classOf[Long]) == src1.getType)
-
-    val map: DataStream[(Integer, String)] = src1.map(x => null)
-    assert(classOf[scala.Tuple2[Integer, String]] == map.getType().getTypeClass)
-
-    val window: DataStream[String] = map
-      .windowAll(GlobalWindows.create())
-      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](5)))
-      .apply((w: GlobalWindow, x: Iterable[(Integer, String)], y: Collector[String]) => {})
-
-    assert(TypeExtractor.getForClass(classOf[String]) == window.getType)
-
-    val flatten: DataStream[Int] = window
-      .windowAll(GlobalWindows.create())
-      .trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](5)))
-      .fold(0, (accumulator: Int, value: String) => 0)
-    assert(TypeExtractor.getForClass(classOf[Int]) == flatten.getType())
-
-    // TODO check for custom case class
-  }
-
-  @Test def operatorTest() {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val src = env.generateSequence(0, 0)
-
-    val mapFunction = new MapFunction[Long, Int] {
-      override def map(value: Long): Int = 0
-    }
-
-    val map = src.map(mapFunction)
-    assert(mapFunction == getFunctionForDataStream(map))
-    assert(getFunctionForDataStream(map.map(x => 0)).isInstanceOf[MapFunction[_, _]])
-    
-    val statefulMap2 = src.keyBy(x => x).mapWithState(
-        (in, state: Option[Long]) => (in, None.asInstanceOf[Option[Long]]))
-    
-    val flatMapFunction = new FlatMapFunction[Long, Int] {
-      override def flatMap(value: Long, out: Collector[Int]): Unit = {}
-    }
-    
-    val flatMap = src.flatMap(flatMapFunction)
-    assert(flatMapFunction == getFunctionForDataStream(flatMap))
-    assert(
-      getFunctionForDataStream(flatMap
-        .flatMap((x: Int, out: Collector[Int]) => {}))
-        .isInstanceOf[FlatMapFunction[_, _]])
-
-    val statefulfMap2 = src.keyBy(x => x).flatMapWithState(
-        (in, state: Option[Long]) => (List(in), None.asInstanceOf[Option[Long]]))
-   
-    val filterFunction = new FilterFunction[Int] {
-      override def filter(value: Int): Boolean = false
-    }
-
-    val unionFilter = map.union(flatMap).filter(filterFunction)
-    assert(filterFunction == getFunctionForDataStream(unionFilter))
-    assert(
-      getFunctionForDataStream(map
-        .filter((x: Int) => true))
-        .isInstanceOf[FilterFunction[_]])
-
-    val statefulFilter2 = src.keyBy( x => x).filterWithState[Long](
-        (in, state: Option[Long]) => (false, None))
-   
-    try {
-      env.getStreamGraph.getStreamEdge(map.getId, unionFilter.getId)
-    }
-    catch {
-      case e: Throwable => {
-        fail(e.getMessage)
-      }
-    }
-
-    try {
-      env.getStreamGraph.getStreamEdge(flatMap.getId, unionFilter.getId)
-    }
-    catch {
-      case e: Throwable => {
-        fail(e.getMessage)
-      }
-    }
-
-    val outputSelector = new OutputSelector[Int] {
-      override def select(value: Int): lang.Iterable[String] = null
-    }
-
-    val split = unionFilter.split(outputSelector)
-    split.print()
-    val outputSelectors = env.getStreamGraph.getStreamNode(unionFilter.getId).getOutputSelectors
-    assert(1 == outputSelectors.size)
-    assert(outputSelector == outputSelectors.get(0))
-
-    unionFilter.split(x => List("a")).print()
-    val moreOutputSelectors = env.getStreamGraph.getStreamNode(unionFilter.getId).getOutputSelectors
-    assert(2 == moreOutputSelectors.size)
-
-    val select = split.select("a")
-    val sink = select.print()
-    val splitEdge =
-      env.getStreamGraph.getStreamEdge(unionFilter.getId, sink.getTransformation.getId)
-    assert("a" == splitEdge.getSelectedNames.get(0))
-
-    val foldFunction = new FoldFunction[Int, String] {
-      override def fold(accumulator: String, value: Int): String = ""
-    }
-    val fold = map.keyBy(x=>x).fold("", foldFunction)
-    assert(foldFunction == getFunctionForDataStream(fold))
-    assert(
-      getFunctionForDataStream(map.keyBy(x=>x)
-        .fold("", (x: String, y: Int) => ""))
-        .isInstanceOf[FoldFunction[_, _]])
-
-    val connect = fold.connect(flatMap)
-
-    val coMapFunction =
-      new CoMapFunction[String, Int, String] {
-        override def map1(value: String): String = ""
-
-        override def map2(value: Int): String = ""
-      }
-    val coMap = connect.map(coMapFunction)
-    assert(coMapFunction == getFunctionForDataStream(coMap))
-
-    try {
-      env.getStreamGraph.getStreamEdge(fold.getId, coMap.getId)
-    }
-    catch {
-      case e: Throwable => {
-        fail(e.getMessage)
-      }
-    }
-    try {
-      env.getStreamGraph.getStreamEdge(flatMap.getId, coMap.getId)
-    }
-    catch {
-      case e: Throwable => {
-        fail(e.getMessage)
-      }
-    }
-  }
-
-  @Test
-  def testChannelSelectors() {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val src = env.generateSequence(0, 0)
-
-    val broadcast = src.broadcast
-    val broadcastSink = broadcast.print()
-    val broadcastPartitioner = env.getStreamGraph
-      .getStreamEdge(src.getId, broadcastSink.getTransformation.getId).getPartitioner
-    assert(broadcastPartitioner.isInstanceOf[BroadcastPartitioner[_]])
-
-    val shuffle: DataStream[Long] = src.shuffle
-    val shuffleSink = shuffle.print()
-    val shufflePartitioner = env.getStreamGraph
-      .getStreamEdge(src.getId, shuffleSink.getTransformation.getId).getPartitioner
-    assert(shufflePartitioner.isInstanceOf[ShufflePartitioner[_]])
-
-    val forward: DataStream[Long] = src.forward
-    val forwardSink = forward.print()
-    val forwardPartitioner = env.getStreamGraph
-      .getStreamEdge(src.getId, forwardSink.getTransformation.getId).getPartitioner
-    assert(forwardPartitioner.isInstanceOf[ForwardPartitioner[_]])
-
-    val rebalance: DataStream[Long] = src.rebalance
-    val rebalanceSink = rebalance.print()
-    val rebalancePartitioner = env.getStreamGraph
-      .getStreamEdge(src.getId, rebalanceSink.getTransformation.getId).getPartitioner
-    assert(rebalancePartitioner.isInstanceOf[RebalancePartitioner[_]])
-
-    val global: DataStream[Long] = src.global
-    val globalSink = global.print()
-    val globalPartitioner = env.getStreamGraph
-      .getStreamEdge(src.getId, globalSink.getTransformation.getId).getPartitioner
-    assert(globalPartitioner.isInstanceOf[GlobalPartitioner[_]])
-  }
-
-  @Test
-  def testIterations() {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    // we need to rebalance before iteration
-    val source = env.fromElements(1, 2, 3).map { t: Int => t }
-
-    val iterated = source.iterate((input: ConnectedStreams[Int, String]) => {
-      val head = input.map(i => (i + 1).toString, s => s)
-      (head.filter(_ == "2"), head.filter(_ != "2"))
-    }, 1000).print()
-
-    val iterated2 = source.iterate((input: DataStream[Int]) => 
-      (input.map(_ + 1), input.map(_.toString)), 2000)
-
-    try {
-      val invalid = source.iterate((input: ConnectedStreams[Int, String]) => {
-        val head = input.partitionByHash(1, 1).map(i => (i + 1).toString, s => s)
-        (head.filter(_ == "2"), head.filter(_ != "2"))
-      }, 1000).print()
-      fail()
-    } catch {
-      case uoe: UnsupportedOperationException =>
-      case e: Exception => fail()
-    }
-
-    val sg = env.getStreamGraph
-
-    assert(sg.getIterationSourceSinkPairs().size() == 2)
-  }
-
-  /////////////////////////////////////////////////////////////
-  // Utilities
-  /////////////////////////////////////////////////////////////
-
-  private def getFunctionForDataStream(dataStream: DataStream[_]): Function = {
-    dataStream.print()
-    val operator = getOperatorForDataStream(dataStream)
-      .asInstanceOf[AbstractUdfStreamOperator[_, _]]
-    operator.getUserFunction.asInstanceOf[Function]
-  }
-
-  private def getOperatorForDataStream(dataStream: DataStream[_]): StreamOperator[_] = {
-    dataStream.print()
-    val env = dataStream.getJavaStream.getExecutionEnvironment
-    val streamGraph: StreamGraph = env.getStreamGraph
-    streamGraph.getStreamNode(dataStream.getId).getOperator
-  }
-
-  private def isPartitioned(edge: StreamEdge): Boolean = {
-    edge.getPartitioner.isInstanceOf[HashPartitioner[_]]
-  }
-
-  private def isCustomPartitioned(edge: StreamEdge): Boolean = {
-    edge.getPartitioner.isInstanceOf[CustomPartitionerWrapper[_, _]]
-  }
-
-  private def createDownStreamId(dataStream: DataStream[_]): Integer = {
-    dataStream.print().getTransformation.getId
-  }
-
-  private def createDownStreamId(dataStream: ConnectedStreams[_, _]): Integer = {
-    val m = dataStream.map(x => 0, x => 0)
-    m.print()
-    m.getId
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
deleted file mode 100644
index e09f164..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.streaming.util.SocketOutputTestBase.DummyStringSchema
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.apache.flink.streaming.util.serialization.JavaDefaultStringSchema
-import org.apache.flink.test.util.MultipleProgramsTestBase
-
-import scala.language.existentials
-
-/**
- * Test programs for built in output formats. Invoked from {@link OutputFormatTest}.
- */
-object OutputFormatTestPrograms {
-
-  def wordCountToText(input : String, outputPath : String) : Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    //Create streams for names and ages by mapping the inputs to the corresponding objects
-    val text = env.fromElements(input)
-    val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
-      .map { (_, 1) }
-      .keyBy(0)
-      .sum(1)
-
-    counts.writeAsText(outputPath)
-
-    env.execute("Scala WordCountToText")
-  }
-
-  def wordCountToCsv(input : String, outputPath : String) : Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    //Create streams for names and ages by mapping the inputs to the corresponding objects
-    val text = env.fromElements(input)
-    val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
-      .map { (_, 1) }
-      .keyBy(0)
-      .sum(1)
-
-    counts.writeAsCsv(outputPath)
-
-    env.execute("Scala WordCountToCsv")
-  }
-
-  def wordCountToSocket(input : String, outputHost : String, outputPort : Int) : Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    //Create streams for names and ages by mapping the inputs to the corresponding objects
-    val text = env.fromElements(input)
-    val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
-      .map { (_, 1) }
-      .keyBy(0)
-      .sum(1)
-      .map(tuple => tuple.toString() + "\n")
-
-    counts.writeToSocket(outputHost, outputPort, new DummyStringSchema())
-
-    env.execute("Scala WordCountToCsv")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala
deleted file mode 100644
index 3342e1e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.runtime.StreamingMode
-import org.apache.flink.streaming.util.TestStreamEnvironment
-import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils}
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.junit.JUnitSuiteLike
-
-trait ScalaStreamingMultipleProgramsTestBase
-  extends TestBaseUtils
-  with  JUnitSuiteLike
-  with BeforeAndAfterAll {
-
-  val parallelism = 4
-  var cluster: Option[ForkableFlinkMiniCluster] = None
-
-  override protected def beforeAll(): Unit = {
-    val cluster = Some(
-      TestBaseUtils.startCluster(
-        1,
-        parallelism,
-        StreamingMode.STREAMING,
-        false,
-        false,
-        true
-      )
-    )
-
-    val clusterEnvironment = new TestStreamEnvironment(cluster.get, parallelism)
-  }
-
-  override protected def afterAll(): Unit = {
-    cluster.foreach {
-      TestBaseUtils.stopCluster(_, TestBaseUtils.DEFAULT_TIMEOUT)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
deleted file mode 100644
index b2e05b3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.scala
-
-import java.util
-
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
-
-/**
- * Test programs for stateful functions.
- */
-object StateTestPrograms {
-
-  def testStatefulFunctions(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    
-    // test stateful map
-    env.generateSequence(0, 10).setParallelism(1)
-      .map { v => (1, v) }.setParallelism(1)
-      .keyBy(_._1)
-      .mapWithState((in, count: Option[Long]) =>
-        count match {
-          case Some(c) => (in._2 - c, Some(c + 1))
-          case None => (in._2, Some(1L))
-        }).setParallelism(1)
-      
-      .addSink(new RichSinkFunction[Long]() {
-        var allZero = true
-        override def invoke(in: Long) = {
-          if (in != 0) allZero = false
-        }
-        override def close() = {
-          assert(allZero)
-        }
-      })
-
-    // test stateful flatmap
-    env.fromElements((1, "First"), (2, "Second"), (1, "Hello world"))
-      .keyBy(_._1)
-      .flatMapWithState((w, s: Option[String]) =>
-        s match {
-          case Some(state) => (w._2.split(" ").toList.map(state + _), Some(w._2))
-          case None => (List(w._2), Some(w._2))
-        })
-      .setParallelism(1)
-      
-      .addSink(new RichSinkFunction[String]() {
-        val received = new util.HashSet[String]()
-        override def invoke(in: String) = { received.add(in) }
-        override def close() = {
-          assert(received.size() == 4)
-          assert(received.contains("First"))
-          assert(received.contains("Second"))
-          assert(received.contains("FirstHello"))
-          assert(received.contains("Firstworld"))
-        }
-      }).setParallelism(1)
-
-    // test stateful filter
-    env.generateSequence(1, 10).keyBy(_ % 2).filterWithState((in, state: Option[Int]) =>
-      state match {
-        case Some(s) => (s < 2, Some(s + 1))
-        case None => (true, Some(1))
-      }).addSink(new RichSinkFunction[Long]() {
-      var numOdd = 0
-      var numEven = 0
-      override def invoke(in: Long) = {
-        if (in % 2 == 0) { numEven += 1 } else { numOdd += 1 }
-      }
-      override def close() = {
-        assert(numOdd == 2)
-        assert(numEven == 2)
-      }
-    }).setParallelism(1)
-
-    env.execute("Stateful test")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala
deleted file mode 100644
index 2131026..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.functions.{RichMapFunction, FoldFunction}
-import org.apache.flink.core.fs.FileSystem
-import org.apache.flink.streaming.api.functions.source.SourceFunction
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit.rules.TemporaryFolder
-import org.junit.{After, Before, Rule, Test}
-
-class StreamingOperatorsITCase extends ScalaStreamingMultipleProgramsTestBase {
-
-  var resultPath1: String = _
-  var resultPath2: String = _
-  var expected1: String = _
-  var expected2: String = _
-
-  val _tempFolder = new TemporaryFolder()
-
-  @Rule
-  def tempFolder: TemporaryFolder = _tempFolder
-
-  @Before
-  def before(): Unit = {
-    val temp = tempFolder
-    resultPath1 = temp.newFile.toURI.toString
-    resultPath2 = temp.newFile.toURI.toString
-    expected1 = ""
-    expected2 = ""
-  }
-
-  @After
-  def after(): Unit = {
-    TestBaseUtils.compareResultsByLinesInMemory(expected1, resultPath1)
-    TestBaseUtils.compareResultsByLinesInMemory(expected2, resultPath2)
-  }
-
-  /** Tests the streaming fold operation. For this purpose a stream of Tuple[Int, Int] is created.
-    * The stream is grouped by the first field. For each group, the resulting stream is folded by
-    * summing up the second tuple field.
-    *
-    */
-  @Test
-  def testFoldOperator(): Unit = {
-    val numElements = 10
-    val numKeys = 2
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    env.setParallelism(2)
-
-    val sourceStream = env.addSource(new SourceFunction[(Int, Int)] {
-
-      override def run(ctx: SourceContext[(Int, Int)]): Unit = {
-        0 until numElements foreach {
-          i => ctx.collect((i % numKeys, i))
-        }
-      }
-
-      override def cancel(): Unit = {}
-    })
-
-    val splittedResult = sourceStream
-      .keyBy(0)
-      .fold(0, new FoldFunction[(Int, Int), Int] {
-        override def fold(accumulator: Int, value: (Int, Int)): Int = {
-          accumulator + value._2
-        }
-      })
-      .map(new RichMapFunction[Int, (Int, Int)] {
-        override def map(value: Int): (Int, Int) = {
-          (getRuntimeContext.getIndexOfThisSubtask, value)
-        }
-      })
-      .split{
-        x =>
-          Seq(x._1.toString)
-      }
-
-    splittedResult
-      .select("0")
-      .map(_._2)
-      .getJavaStream
-      .writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE)
-    splittedResult
-      .select("1")
-      .map(_._2)
-      .getJavaStream
-      .writeAsText(resultPath2, FileSystem.WriteMode.OVERWRITE)
-
-    val groupedSequence = 0 until numElements groupBy( _ % numKeys)
-
-    expected1 = groupedSequence(0).scanLeft(0)(_ + _).tail.mkString("\n")
-    expected2 = groupedSequence(1).scanLeft(0)(_ + _).tail.mkString("\n")
-
-    env.execute()
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 101f3b5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.scala
-
-import java.lang.reflect.Method
-
-import org.apache.flink.api.scala.completeness.ScalaAPICompletenessTestBase
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream}
-
-import scala.language.existentials
-
-import org.junit.Test
-
-/**
- * This checks whether the streaming Scala API is up to feature parity with the Java API.
- * Implements the {@link ScalaAPICompletenessTest} for streaming.
- */
-class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
-
-  override def isExcludedByName(method: Method): Boolean = {
-    val name = method.getDeclaringClass.getName + "." + method.getName
-    val excludedNames = Seq(
-      // These are only used internally. Should be internal API but Java doesn't have
-      // private[flink].
-      "org.apache.flink.streaming.api.datastream.DataStream.getExecutionEnvironment",
-      "org.apache.flink.streaming.api.datastream.DataStream.getType",
-      "org.apache.flink.streaming.api.datastream.DataStream.copy",
-      "org.apache.flink.streaming.api.datastream.DataStream.transform",
-      "org.apache.flink.streaming.api.datastream.DataStream.getTransformation",
-      "org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator.copy",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getExecutionEnvironment",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getExecutionEnvironment",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getFirstInput",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getSecondInput",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getType1",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.getType2",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.addGeneralWindowCombine",
-      "org.apache.flink.streaming.api.datastream.ConnectedStreams.transform",
-
-      "org.apache.flink.streaming.api.datastream.WindowedDataStream.getType",
-      "org.apache.flink.streaming.api.datastream.WindowedDataStream.getExecutionConfig",
-
-      "org.apache.flink.streaming.api.datastream.WindowedStream.getExecutionEnvironment",
-      "org.apache.flink.streaming.api.datastream.WindowedStream.getInputType",
-      "org.apache.flink.streaming.api.datastream.AllWindowedStream.getExecutionEnvironment",
-      "org.apache.flink.streaming.api.datastream.AllWindowedStream.getInputType",
-
-      "org.apache.flink.streaming.api.datastream.KeyedStream.transform",
-      "org.apache.flink.streaming.api.datastream.KeyedStream.getKeySelector",
-
-      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.isChainingEnabled",
-      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment." +
-        "getStateHandleProvider",
-      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getCheckpointInterval",
-      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.addOperator",
-      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getCheckpointingMode",
-      "org.apache.flink.streaming.api.environment.StreamExecutionEnvironment." +
-        "isForceCheckpointing",
-
-
-      // TypeHints are only needed for Java API, Scala API doesn't need them
-      "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
-      """^org\.apache\.flink\.streaming.api.*project""",
-
-      // Cleaning is easier in the Scala API
-      """^org\.apache\.flink\.streaming.api.*clean""",
-
-      // Object methods
-      """^.*notify""",
-      """^.*wait""",
-      """^.*notifyAll""",
-      """^.*equals""",
-      """^.*toString""",
-      """^.*getClass""",
-      """^.*hashCode"""
-    ).map(_.r)
-    lazy val excludedByPattern =
-      excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).nonEmpty
-    name.contains("$") || excludedNames.contains(name) || excludedByPattern
-  }
-
-  @Test
-  override def testCompleteness(): Unit = {
-    checkMethods("DataStream", "DataStream", classOf[JavaStream[_]], classOf[DataStream[_]])
-
-    checkMethods(
-      "StreamExecutionEnvironment", "StreamExecutionEnvironment",
-      classOf[org.apache.flink.streaming.api.environment.StreamExecutionEnvironment],
-      classOf[StreamExecutionEnvironment])
-
-    checkMethods(
-      "SingleOutputStreamOperator", "DataStream",
-      classOf[org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator[_,_]],
-      classOf[DataStream[_]])
-
-    checkMethods(
-      "ConnectedStreams", "ConnectedStreams",
-      classOf[org.apache.flink.streaming.api.datastream.ConnectedStreams[_,_]],
-      classOf[ConnectedStreams[_,_]])
-
-    checkMethods(
-      "SplitStream", "SplitStream",
-      classOf[org.apache.flink.streaming.api.datastream.SplitStream[_]],
-      classOf[SplitStream[_]])
-
-    checkMethods(
-      "WindowedStream", "WindowedStream",
-      classOf[org.apache.flink.streaming.api.datastream.WindowedStream[_, _, _]],
-      classOf[WindowedStream[_, _, _]])
-
-    checkMethods(
-      "AllWindowedStream", "AllWindowedStream",
-      classOf[org.apache.flink.streaming.api.datastream.AllWindowedStream[_, _]],
-      classOf[AllWindowedStream[_, _]])
-
-    checkMethods(
-      "KeyedStream", "KeyedStream",
-      classOf[org.apache.flink.streaming.api.datastream.KeyedStream[_, _]],
-      classOf[KeyedStream[_, _]])
-
-    checkMethods(
-      "JoinedStreams.WithWindow", "JoinedStreams.WithWindow",
-      classOf[org.apache.flink.streaming.api.datastream.JoinedStreams.WithWindow[_,_,_,_]],
-      classOf[JoinedStreams.WithWindow[_,_,_,_]])
-
-    checkMethods(
-      "CoGroupedStreams.WithWindow", "CoGroupedStreams.WithWindow",
-      classOf[org.apache.flink.streaming.api.datastream.CoGroupedStreams.WithWindow[_,_,_,_]],
-      classOf[CoGroupedStreams.WithWindow[_,_,_,_]])
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
deleted file mode 100644
index d4e8bb2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.streaming.api.TimeCharacteristic
-import org.apache.flink.streaming.api.functions.TimestampExtractor
-import org.apache.flink.streaming.api.functions.sink.SinkFunction
-import org.apache.flink.streaming.api.functions.source.SourceFunction
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows
-import org.apache.flink.streaming.api.windowing.time.Time
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Test
-import org.junit.Assert._
-
-import scala.collection.mutable
-
-/**
- * Tests for Folds over windows. These also test whether OutputTypeConfigurable functions
- * work for windows, because FoldWindowFunction is OutputTypeConfigurable.
- */
-class WindowFoldITCase extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testFoldWindow(): Unit = {
-    WindowFoldITCase.testResults = mutable.MutableList()
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    env.setParallelism(1)
-
-    val source1 = env.addSource(new SourceFunction[(String, Int)]() {
-      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
-        ctx.collect(("a", 0))
-        ctx.collect(("a", 1))
-        ctx.collect(("a", 2))
-        ctx.collect(("b", 3))
-        ctx.collect(("b", 4))
-        ctx.collect(("b", 5))
-        ctx.collect(("a", 6))
-        ctx.collect(("a", 7))
-        ctx.collect(("a", 8))
-      }
-
-      def cancel() {
-      }
-    }).assignTimestamps(new WindowFoldITCase.Tuple2TimestampExtractor)
-
-    source1
-      .keyBy(0)
-      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-      .fold(("R:", 0), { (acc: (String, Int), v: (String, Int)) => (acc._1 + v._1, acc._2 + v._2) })
-      .addSink(new SinkFunction[(String, Int)]() {
-        def invoke(value: (String, Int)) {
-        WindowFoldITCase.testResults += value.toString
-        }
-      })
-
-    env.execute("Fold Window Test")
-
-    val expectedResult = mutable.MutableList(
-      "(R:aaa,3)",
-      "(R:aaa,21)",
-      "(R:bbb,12)")
-
-    assertEquals(expectedResult.sorted, WindowFoldITCase.testResults.sorted)
-  }
-
-  @Test
-  def testFoldAllWindow(): Unit = {
-    WindowFoldITCase.testResults = mutable.MutableList()
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    env.setParallelism(1)
-
-    val source1 = env.addSource(new SourceFunction[(String, Int)]() {
-      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
-        ctx.collect(("a", 0))
-        ctx.collect(("a", 1))
-        ctx.collect(("a", 2))
-        ctx.collect(("b", 3))
-        ctx.collect(("a", 3))
-        ctx.collect(("b", 4))
-        ctx.collect(("a", 4))
-        ctx.collect(("b", 5))
-        ctx.collect(("a", 5))
-      }
-
-      def cancel() {
-      }
-    }).assignTimestamps(new WindowFoldITCase.Tuple2TimestampExtractor)
-
-    source1
-      .windowAll(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-      .fold(("R:", 0), { (acc: (String, Int), v: (String, Int)) => (acc._1 + v._1, acc._2 + v._2) })
-      .addSink(new SinkFunction[(String, Int)]() {
-      def invoke(value: (String, Int)) {
-        WindowFoldITCase.testResults += value.toString
-      }
-    })
-
-    env.execute("Fold All-Window Test")
-
-    val expectedResult = mutable.MutableList(
-      "(R:aaa,3)",
-      "(R:bababa,24)")
-
-    assertEquals(expectedResult.sorted, WindowFoldITCase.testResults.sorted)
-  }
-
-}
-
-
-object WindowFoldITCase {
-  private var testResults: mutable.MutableList[String] = null
-
-  private class Tuple2TimestampExtractor extends TimestampExtractor[(String, Int)] {
-    def extractTimestamp(element: (String, Int), currentTimestamp: Long): Long = {
-      element._2
-    }
-
-    def extractWatermark(element: (String, Int), currentTimestamp: Long): Long = {
-      element._2 - 1
-    }
-
-    def getCurrentWatermark: Long = {
-      Long.MinValue
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
deleted file mode 100644
index 46981ab..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
+++ /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.scala
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction
-import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.streaming.api.windowing.assigners.{TumblingTimeWindows, SlidingTimeWindows}
-import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
-import org.apache.flink.streaming.api.windowing.time.Time
-import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, CountTrigger}
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.{HeapWindowBuffer, PreAggregatingHeapWindowBuffer}
-import org.apache.flink.streaming.runtime.operators.windowing.{EvictingWindowOperator, WindowOperator, AccumulatingProcessingTimeWindowOperator, AggregatingProcessingTimeWindowOperator}
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.apache.flink.util.Collector
-
-import org.junit.Assert._
-import org.junit.Test
-
-class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
-
-  /**
-   * These tests ensure that the fast aligned time windows operator is used if the
-   * conditions are right.
-   */
-  @Test
-  def testFastTimeWindows(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .keyBy(0)
-      .window(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .reduce(reducer)
-
-    val transform1 = window1.getJavaStream.getTransformation
-        .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-    
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[AggregatingProcessingTimeWindowOperator[_, _]])
-
-    val window2 = source
-      .keyBy(0)
-      .window(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-        def apply(
-            key: Tuple,
-            window: TimeWindow,
-            values: java.lang.Iterable[(String, Int)],
-            out: Collector[(String, Int)]) { }
-      })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[AccumulatingProcessingTimeWindowOperator[_, _, _]])
-  }
-
-  @Test
-  def testNonEvicting(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .keyBy(0)
-      .window(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
-      .reduce(reducer)
-
-    val transform1 = window1.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[WindowOperator[_, _, _, _]])
-    val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _]]
-    assertTrue(winOperator1.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
-    assertTrue(
-      winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
-
-
-    val window2 = source
-      .keyBy(0)
-      .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
-      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-      def apply(
-                    tuple: Tuple,
-                    window: TimeWindow,
-                    values: java.lang.Iterable[(String, Int)],
-                    out: Collector[(String, Int)]) { }
-    })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[WindowOperator[_, _, _, _]])
-    val winOperator2 = operator2.asInstanceOf[WindowOperator[_, _, _, _]]
-    assertTrue(winOperator2.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
-    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
-  }
-
-  @Test
-  def testEvicting(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .keyBy(0)
-      .window(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
-      .reduce(reducer)
-
-    val transform1 = window1.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[EvictingWindowOperator[_, _, _, _]])
-    val winOperator1 = operator1.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
-    assertTrue(winOperator1.getTrigger.isInstanceOf[ProcessingTimeTrigger])
-    assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
-    assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
-
-
-    val window2 = source
-      .keyBy(0)
-      .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
-      .evictor(CountEvictor.of(1000))
-      .apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-      def apply(
-                    tuple: Tuple,
-                    window: TimeWindow,
-                    values: java.lang.Iterable[(String, Int)],
-                    out: Collector[(String, Int)]) { }
-    })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[EvictingWindowOperator[_, _, _, _]])
-    val winOperator2 = operator2.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
-    assertTrue(winOperator2.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
-    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
-  }
-
-  @Test
-  def testPreReduce(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .keyBy(0)
-      .window(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
-      .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-        def apply(
-                   tuple: Tuple,
-                   window: TimeWindow,
-                   values: java.lang.Iterable[(String, Int)],
-                   out: Collector[(String, Int)]) { }
-      })
-
-    val transform1 = window1.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[WindowOperator[_, _, _, _]])
-    val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _]]
-    assertTrue(winOperator1.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
-    assertTrue(
-      winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
-
-
-    val window2 = source
-      .keyBy(0)
-      .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
-      .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-        def apply(
-                   tuple: Tuple,
-                   window: TimeWindow,
-                   values: java.lang.Iterable[(String, Int)],
-                   out: Collector[(String, Int)]) { }
-      })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[WindowOperator[_, _, _, _]])
-    val winOperator2 = operator2.asInstanceOf[WindowOperator[_, _, _, _]]
-    assertTrue(winOperator2.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
-    assertTrue(
-      winOperator2.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/pom.xml b/flink-staging/flink-streaming/pom.xml
deleted file mode 100644
index aa233c8..0000000
--- a/flink-staging/flink-streaming/pom.xml
+++ /dev/null
@@ -1,43 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-staging</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-streaming-parent</artifactId>
-	<name>flink-streaming</name>
-	<packaging>pom</packaging>
-
-	<modules>
-		<module>flink-streaming-core</module>
-		<module>flink-streaming-scala</module>
-		<module>flink-streaming-examples</module>
-		<module>flink-streaming-connectors</module>
-	</modules>
-	
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/pom.xml b/flink-staging/pom.xml
index 7bc76a7..8483bfa 100644
--- a/flink-staging/pom.xml
+++ b/flink-staging/pom.xml
@@ -38,7 +38,6 @@ under the License.
 		<module>flink-avro</module>
 		<module>flink-jdbc</module>
 		<module>flink-hadoop-compatibility</module>
-		<module>flink-streaming</module>
 		<module>flink-hbase</module>
 		<module>flink-hcatalog</module>
 		<module>flink-table</module>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml b/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
new file mode 100644
index 0000000..f3efe2e
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/pom.xml
@@ -0,0 +1,106 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-streaming-connectors-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-elasticsearch</artifactId>
+	<name>flink-connector-elasticsearch</name>
+
+	<packaging>jar</packaging>
+
+	<!-- Allow users to pass custom connector versions -->
+	<properties>
+		<elasticsearch.version>1.7.1</elasticsearch.version>
+	</properties>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+        <dependency>
+            <groupId>org.elasticsearch</groupId>
+            <artifactId>elasticsearch</artifactId>
+            <version>${elasticsearch.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${guava.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-streaming-java</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-tests</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<rerunFailingTestsCount>3</rerunFailingTestsCount>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-failsafe-plugin</artifactId>
+				<configuration>
+					<rerunFailingTestsCount>3</rerunFailingTestsCount>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java b/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
new file mode 100644
index 0000000..546ec8d
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.connectors.elasticsearch;
+
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.action.bulk.BulkProcessor;
+import org.elasticsearch.action.bulk.BulkRequest;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.collect.ImmutableList;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.node.Node;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
+
+
+/**
+ * Sink that emits its input elements to an Elasticsearch cluster.
+ *
+ * <p>
+ * When using the first constructor {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)}
+ * the sink will create a local {@link Node} for communicating with the
+ * Elasticsearch cluster. When using the second constructor
+ * {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)} a {@link TransportClient} will
+ * be used instead.
+ *
+ * <p>
+ * <b>Attention: </b> When using the {@code TransportClient} the sink will fail if no cluster
+ * can be connected to. With the {@code Node Client} the sink will block and wait for a cluster
+ * to come online.
+ *
+ * <p>
+ * The {@link Map} passed to the constructor is forwarded to Elasticsearch when creating
+ * the {@link Node} or {@link TransportClient}. The config keys can be found in the Elasticsearch
+ * documentation. An important setting is {@code cluster.name}, this should be set to the name
+ * of the cluster that the sink should emit to.
+ *
+ * <p>
+ * Internally, the sink will use a {@link BulkProcessor} to send {@link IndexRequest IndexRequests}.
+ * This will buffer elements before sending a request to the cluster. The behaviour of the
+ * {@code BulkProcessor} can be configured using these config keys:
+ * <ul>
+ *   <li> {@code bulk.flush.max.actions}: Maximum amount of elements to buffer
+ *   <li> {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer
+ *   <li> {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two
+ *   settings in milliseconds
+ * </ul>
+ *
+ * <p>
+ * You also have to provide an {@link IndexRequestBuilder}. This is used to create an
+ * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
+ * {@link org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder} for an example.
+ *
+ * @param <T> Type of the elements emitted by this sink
+ */
+public class ElasticsearchSink<T> extends RichSinkFunction<T> {
+
+	public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions";
+	public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb";
+	public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms";
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class);
+
+	/**
+	 * The user specified config map that we forward to Elasticsearch when we create the Client.
+	 */
+	private final Map<String, String> userConfig;
+
+	/**
+	 * The list of nodes that the TransportClient should connect to. This is null if we are using
+	 * an embedded Node to get a Client.
+	 */
+	private final List<TransportAddress> transportNodes;
+
+	/**
+	 * The builder that is used to construct an {@link IndexRequest} from the incoming element.
+	 */
+	private final IndexRequestBuilder<T> indexRequestBuilder;
+
+	/**
+	 * The embedded Node that is used to communicate with the Elasticsearch cluster. This is null
+	 * if we are using a TransportClient.
+	 */
+	private transient Node node;
+
+	/**
+	 * The Client that was either retrieved from a Node or is a TransportClient.
+	 */
+	private transient Client client;
+
+	/**
+	 * Bulk processor that was created using the client
+	 */
+	private transient BulkProcessor bulkProcessor;
+
+	/**
+	 * This is set from inside the BulkProcessor listener if there where failures in processing.
+	 */
+	private final AtomicBoolean hasFailure = new AtomicBoolean(false);
+
+	/**
+	 * This is set from inside the BulkProcessor listener if a Throwable was thrown during processing.
+	 */
+	private final AtomicReference<Throwable> failureThrowable = new AtomicReference<>();
+
+	/**
+	 * Creates a new ElasticsearchSink that connects to the cluster using an embedded Node.
+	 *
+	 * @param userConfig The map of user settings that are passed when constructing the Node and BulkProcessor
+	 * @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
+	 */
+	public ElasticsearchSink(Map<String, String> userConfig, IndexRequestBuilder<T> indexRequestBuilder) {
+		this.userConfig = userConfig;
+		this.indexRequestBuilder = indexRequestBuilder;
+		transportNodes = null;
+	}
+
+	/**
+	 * Creates a new ElasticsearchSink that connects to the cluster using a TransportClient.
+	 *
+	 * @param userConfig The map of user settings that are passed when constructing the TransportClient and BulkProcessor
+	 * @param transportNodes The Elasticsearch Nodes to which to connect using a {@code TransportClient}
+	 * @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
+	 *
+	 */
+	public ElasticsearchSink(Map<String, String> userConfig, List<TransportAddress> transportNodes, IndexRequestBuilder<T> indexRequestBuilder) {
+		this.userConfig = userConfig;
+		this.indexRequestBuilder = indexRequestBuilder;
+		this.transportNodes = transportNodes;
+	}
+
+	/**
+	 * Initializes the connection to Elasticsearch by either creating an embedded
+	 * {@link org.elasticsearch.node.Node} and retrieving the
+	 * {@link org.elasticsearch.client.Client} from it or by creating a
+	 * {@link org.elasticsearch.client.transport.TransportClient}.
+	 */
+	@Override
+	public void open(Configuration configuration) {
+		if (transportNodes == null) {
+			// Make sure that we disable http access to our embedded node
+			Settings settings =
+					ImmutableSettings.settingsBuilder()
+							.put(userConfig)
+							.put("http.enabled", false)
+							.build();
+
+			node =
+					nodeBuilder()
+							.settings(settings)
+							.client(true)
+							.data(false)
+							.node();
+
+			client = node.client();
+
+			if (LOG.isInfoEnabled()) {
+				LOG.info("Created Elasticsearch Client {} from embedded Node", client);
+			}
+
+		} else {
+			Settings settings = ImmutableSettings.settingsBuilder()
+					.put(userConfig)
+					.build();
+
+			TransportClient transportClient = new TransportClient(settings);
+			for (TransportAddress transport: transportNodes) {
+				transportClient.addTransportAddress(transport);
+			}
+
+			// verify that we actually are connected to a cluster
+			ImmutableList<DiscoveryNode> nodes = transportClient.connectedNodes();
+			if (nodes.isEmpty()) {
+				throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Connected to nodes: " + nodes.toString());
+				}
+			}
+
+			client = transportClient;
+
+			if (LOG.isInfoEnabled()) {
+				LOG.info("Created Elasticsearch TransportClient {}", client);
+			}
+		}
+
+		BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder(
+				client,
+				new BulkProcessor.Listener() {
+					@Override
+					public void beforeBulk(long executionId,
+							BulkRequest request) {
+
+					}
+
+					@Override
+					public void afterBulk(long executionId,
+							BulkRequest request,
+							BulkResponse response) {
+						if (response.hasFailures()) {
+							for (BulkItemResponse itemResp : response.getItems()) {
+								if (itemResp.isFailed()) {
+									LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage());
+									failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage()));
+								}
+							}
+							hasFailure.set(true);
+						}
+					}
+
+					@Override
+					public void afterBulk(long executionId,
+							BulkRequest request,
+							Throwable failure) {
+						LOG.error(failure.getMessage());
+						failureThrowable.compareAndSet(null, failure);
+						hasFailure.set(true);
+					}
+				});
+
+		// This makes flush() blocking
+		bulkProcessorBuilder.setConcurrentRequests(0);
+
+		ParameterTool params = ParameterTool.fromMap(userConfig);
+
+		if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) {
+			bulkProcessorBuilder.setBulkActions(params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS));
+		}
+
+		if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) {
+			bulkProcessorBuilder.setBulkSize(new ByteSizeValue(params.getInt(
+					CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB), ByteSizeUnit.MB));
+		}
+
+		if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) {
+			bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)));
+		}
+
+		bulkProcessor = bulkProcessorBuilder.build();
+	}
+
+	@Override
+	public void invoke(T element) {
+		IndexRequest indexRequest = indexRequestBuilder.createIndexRequest(element, getRuntimeContext());
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Emitting IndexRequest: {}", indexRequest);
+		}
+
+		bulkProcessor.add(indexRequest);
+	}
+
+	@Override
+	public void close() {
+		if (bulkProcessor != null) {
+			bulkProcessor.close();
+			bulkProcessor = null;
+		}
+
+		if (client != null) {
+			client.close();
+		}
+
+		if (node != null) {
+			node.close();
+		}
+
+		if (hasFailure.get()) {
+			Throwable cause = failureThrowable.get();
+			if (cause != null) {
+				throw new RuntimeException("An error occured in ElasticsearchSink.", cause);
+			} else {
+				throw new RuntimeException("An error occured in ElasticsearchSink.");
+
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java b/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
new file mode 100644
index 0000000..04ae40a
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.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.connectors.elasticsearch;
+
+import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.elasticsearch.action.index.IndexRequest;
+
+import java.io.Serializable;
+
+/**
+ * Function that creates an {@link IndexRequest} from an element in a Stream.
+ *
+ * <p>
+ * This is used by {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink}
+ * to prepare elements for sending them to Elasticsearch. See
+ * <a href="https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/index_.html">Index API</a>
+ * for information about how to format data for adding it to an Elasticsearch index.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre>{@code
+ *     private static class MyIndexRequestBuilder implements IndexRequestBuilder<String> {
+ *
+ *         public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
+ *             Map<String, Object> json = new HashMap<>();
+ *             json.put("data", element);
+ *
+ *             return Requests.indexRequest()
+ *                 .index("my-index")
+ *                 .type("my-type")
+ *                 .source(json);
+ *         }
+ *     }
+ * }</pre>
+ *
+ * @param <T> The type of the element handled by this {@code IndexRequestBuilder}
+ */
+public interface IndexRequestBuilder<T> extends Function, Serializable {
+
+	/**
+	 * Creates an {@link org.elasticsearch.action.index.IndexRequest} from an element.
+	 *
+	 * @param element The element that needs to be turned in to an {@code IndexRequest}
+	 * @param ctx The Flink {@link RuntimeContext} of the {@link ElasticsearchSink}
+	 *
+	 * @return The constructed {@code IndexRequest}
+	 */
+	IndexRequest createIndexRequest(T element, RuntimeContext ctx);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java b/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
new file mode 100644
index 0000000..298eb64
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.elasticsearch.examples;
+
+import com.google.common.collect.Maps;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink;
+import org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.client.Requests;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that
+ * you have a cluster names "elasticsearch" running or change the cluster name in the config map.
+ */
+public class ElasticsearchExample {
+
+	public static void main(String[] args) throws Exception {
+		
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		DataStreamSource<String> source = env.addSource(new SourceFunction<String>() {
+			private static final long serialVersionUID = 1L;
+
+			private volatile boolean running = true;
+
+			@Override
+			public void run(SourceContext<String> ctx) throws Exception {
+				for (int i = 0; i < 20 && running; i++) {
+					ctx.collect("message #" + i);
+				}
+			}
+
+			@Override
+			public void cancel() {
+				running = false;
+			}
+		});
+
+		Map<String, String> config = Maps.newHashMap();
+		// This instructs the sink to emit after every element, otherwise they would be buffered
+		config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+
+		source.addSink(new ElasticsearchSink<>(config, new IndexRequestBuilder<String>() {
+			@Override
+			public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
+				Map<String, Object> json = new HashMap<>();
+				json.put("data", element);
+
+				return Requests.indexRequest()
+						.index("my-index")
+						.type("my-type")
+						.source(json);
+			}
+		}));
+
+
+		env.execute("Elasticsearch Example");
+	}
+}


[28/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
deleted file mode 100644
index c23a4f4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ /dev/null
@@ -1,694 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.List;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.ConnectedStreams;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper;
-import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.util.NoOpSink;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-public class DataStreamTest extends StreamingMultipleProgramsTestBase {
-
-
-	/**
-	 * Tests {@link SingleOutputStreamOperator#name(String)} functionality.
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void testNaming() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Long> dataStream1 = env.generateSequence(0, 0).name("testSource1")
-				.map(new MapFunction<Long, Long>() {
-					@Override
-					public Long map(Long value) throws Exception {
-						return null;
-					}
-				}).name("testMap");
-
-		DataStream<Long> dataStream2 = env.generateSequence(0, 0).name("testSource2")
-				.map(new MapFunction<Long, Long>() {
-					@Override
-					public Long map(Long value) throws Exception {
-						return null;
-					}
-				}).name("testMap");
-
-		DataStreamSink<Long> connected = dataStream1.connect(dataStream2)
-				.flatMap(new CoFlatMapFunction<Long, Long, Long>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void flatMap1(Long value, Collector<Long> out) throws Exception {
-					}
-
-					@Override
-					public void flatMap2(Long value, Collector<Long> out) throws Exception {
-					}
-				}).name("testCoFlatMap")
-				.windowAll(GlobalWindows.create())
-				.trigger(PurgingTrigger.of(CountTrigger.of(10)))
-				.fold(0L, new FoldFunction<Long, Long>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public Long fold(Long accumulator, Long value) throws Exception {
-						return null;
-					}
-				})
-				.name("testWindowFold")
-				.print();
-
-		//test functionality through the operator names in the execution plan
-		String plan = env.getExecutionPlan();
-
-		assertTrue(plan.contains("testSource1"));
-		assertTrue(plan.contains("testSource2"));
-		assertTrue(plan.contains("testMap"));
-		assertTrue(plan.contains("testMap"));
-		assertTrue(plan.contains("testCoFlatMap"));
-		assertTrue(plan.contains("testWindowFold"));
-	}
-
-	/**
-	 * Tests that {@link DataStream#keyBy} and {@link DataStream#partitionByHash} result in
-	 * different and correct topologies. Does the some for the {@link ConnectedStreams}.
-	 */
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testPartitioning() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Tuple2<Long, Long>> src1 = env.fromElements(new Tuple2<>(0L, 0L));
-		DataStream<Tuple2<Long, Long>> src2 = env.fromElements(new Tuple2<>(0L, 0L));
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connected = src1.connect(src2);
-
-		//Testing DataStream grouping
-		DataStream<Tuple2<Long, Long>> group1 = src1.keyBy(0);
-		DataStream<Tuple2<Long, Long>> group2 = src1.keyBy(1, 0);
-		DataStream<Tuple2<Long, Long>> group3 = src1.keyBy("f0");
-		DataStream<Tuple2<Long, Long>> group4 = src1.keyBy(new FirstSelector());
-
-		int id1 = createDownStreamId(group1);
-		int id2 = createDownStreamId(group2);
-		int id3 = createDownStreamId(group3);
-		int id4 = createDownStreamId(group4);
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id1)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id2)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id3)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id4)));
-
-		assertTrue(isKeyed(group1));
-		assertTrue(isKeyed(group2));
-		assertTrue(isKeyed(group3));
-		assertTrue(isKeyed(group4));
-
-		//Testing DataStream partitioning
-		DataStream<Tuple2<Long, Long>> partition1 = src1.partitionByHash(0);
-		DataStream<Tuple2<Long, Long>> partition2 = src1.partitionByHash(1, 0);
-		DataStream<Tuple2<Long, Long>> partition3 = src1.partitionByHash("f0");
-		DataStream<Tuple2<Long, Long>> partition4 = src1.partitionByHash(new FirstSelector());
-
-		int pid1 = createDownStreamId(partition1);
-		int pid2 = createDownStreamId(partition2);
-		int pid3 = createDownStreamId(partition3);
-		int pid4 = createDownStreamId(partition4);
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid1)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid2)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid3)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid4)));
-
-		assertFalse(isKeyed(partition1));
-		assertFalse(isKeyed(partition3));
-		assertFalse(isKeyed(partition2));
-		assertFalse(isKeyed(partition4));
-
-		// Testing DataStream custom partitioning
-		Partitioner<Long> longPartitioner = new Partitioner<Long>() {
-			@Override
-			public int partition(Long key, int numPartitions) {
-				return 100;
-			}
-		};
-
-		DataStream<Tuple2<Long, Long>> customPartition1 = src1.partitionCustom(longPartitioner, 0);
-		DataStream<Tuple2<Long, Long>> customPartition3 = src1.partitionCustom(longPartitioner, "f0");
-		DataStream<Tuple2<Long, Long>> customPartition4 = src1.partitionCustom(longPartitioner, new FirstSelector());
-
-		int cid1 = createDownStreamId(customPartition1);
-		int cid2 = createDownStreamId(customPartition3);
-		int cid3 = createDownStreamId(customPartition4);
-
-		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid1)));
-		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid2)));
-		assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid3)));
-
-		assertFalse(isKeyed(customPartition1));
-		assertFalse(isKeyed(customPartition3));
-		assertFalse(isKeyed(customPartition4));
-
-		//Testing ConnectedStreams grouping
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup1 = connected.keyBy(0, 0);
-		Integer downStreamId1 = createDownStreamId(connectedGroup1);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup2 = connected.keyBy(new int[]{0}, new int[]{0});
-		Integer downStreamId2 = createDownStreamId(connectedGroup2);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup3 = connected.keyBy("f0", "f0");
-		Integer downStreamId3 = createDownStreamId(connectedGroup3);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup4 = connected.keyBy(new String[]{"f0"}, new String[]{"f0"});
-		Integer downStreamId4 = createDownStreamId(connectedGroup4);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedGroup5 = connected.keyBy(new FirstSelector(), new FirstSelector());
-		Integer downStreamId5 = createDownStreamId(connectedGroup5);
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId1)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId1)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId2)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId2)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId3)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId3)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId4)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId4)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId5)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId5)));
-
-		assertTrue(isKeyed(connectedGroup1));
-		assertTrue(isKeyed(connectedGroup2));
-		assertTrue(isKeyed(connectedGroup3));
-		assertTrue(isKeyed(connectedGroup4));
-		assertTrue(isKeyed(connectedGroup5));
-
-		//Testing ConnectedStreams partitioning
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition1 = connected.partitionByHash(0, 0);
-		Integer connectDownStreamId1 = createDownStreamId(connectedPartition1);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition2 = connected.partitionByHash(new int[]{0}, new int[]{0});
-		Integer connectDownStreamId2 = createDownStreamId(connectedPartition2);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition3 = connected.partitionByHash("f0", "f0");
-		Integer connectDownStreamId3 = createDownStreamId(connectedPartition3);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition4 = connected.partitionByHash(new String[]{"f0"}, new String[]{"f0"});
-		Integer connectDownStreamId4 = createDownStreamId(connectedPartition4);
-
-		ConnectedStreams<Tuple2<Long, Long>, Tuple2<Long, Long>> connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector());
-		Integer connectDownStreamId5 = createDownStreamId(connectedPartition5);
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
-				connectDownStreamId1)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
-				connectDownStreamId1)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
-				connectDownStreamId2)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
-				connectDownStreamId2)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
-				connectDownStreamId3)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
-				connectDownStreamId3)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
-				connectDownStreamId4)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
-				connectDownStreamId4)));
-
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(),
-				connectDownStreamId5)));
-		assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(),
-				connectDownStreamId5)));
-
-		assertFalse(isKeyed(connectedPartition1));
-		assertFalse(isKeyed(connectedPartition2));
-		assertFalse(isKeyed(connectedPartition3));
-		assertFalse(isKeyed(connectedPartition4));
-		assertFalse(isKeyed(connectedPartition5));
-	}
-
-	/**
-	 * Tests whether parallelism gets set.
-	 */
-	@Test
-	public void testParallelism() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Tuple2<Long, Long>> src = env.fromElements(new Tuple2<>(0L, 0L));
-		env.setParallelism(10);
-
-		SingleOutputStreamOperator<Long, ?> map = src.map(new MapFunction<Tuple2<Long, Long>, Long>() {
-			@Override
-			public Long map(Tuple2<Long, Long> value) throws Exception {
-				return null;
-			}
-		}).name("MyMap");
-
-		DataStream<Long> windowed = map
-				.windowAll(GlobalWindows.create())
-				.trigger(PurgingTrigger.of(CountTrigger.of(10)))
-				.fold(0L, new FoldFunction<Long, Long>() {
-					@Override
-					public Long fold(Long accumulator, Long value) throws Exception {
-						return null;
-					}
-				});
-
-		windowed.addSink(new NoOpSink<Long>());
-
-		DataStreamSink<Long> sink = map.addSink(new SinkFunction<Long>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void invoke(Long value) throws Exception {
-			}
-		});
-
-		assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism());
-		assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism());
-		assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism());
-		assertEquals(10,
-				env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism());
-
-		env.setParallelism(7);
-
-		// Some parts, such as windowing rely on the fact that previous operators have a parallelism
-		// set when instantiating the Discretizer. This would break if we dynamically changed
-		// the parallelism of operations when changing the setting on the Execution Environment.
-		assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism());
-		assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism());
-		assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism());
-		assertEquals(10, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism());
-
-		try {
-			src.setParallelism(3);
-			fail();
-		} catch (IllegalArgumentException success) {
-			// do nothing
-		}
-
-		DataStreamSource<Long> parallelSource = env.generateSequence(0, 0);
-		parallelSource.addSink(new NoOpSink<Long>());
-		assertEquals(7, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism());
-
-		parallelSource.setParallelism(3);
-		assertEquals(3, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism());
-
-		map.setParallelism(2);
-		assertEquals(2, env.getStreamGraph().getStreamNode(map.getId()).getParallelism());
-
-		sink.setParallelism(4);
-		assertEquals(4, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism());
-	}
-
-	@Test
-	public void testTypeInfo() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Long> src1 = env.generateSequence(0, 0);
-		assertEquals(TypeExtractor.getForClass(Long.class), src1.getType());
-
-		DataStream<Tuple2<Integer, String>> map = src1.map(new MapFunction<Long, Tuple2<Integer, String>>() {
-			@Override
-			public Tuple2<Integer, String> map(Long value) throws Exception {
-				return null;
-			}
-		});
-
-		assertEquals(TypeExtractor.getForObject(new Tuple2<>(0, "")), map.getType());
-
-		DataStream<String> window = map
-				.windowAll(GlobalWindows.create())
-				.trigger(PurgingTrigger.of(CountTrigger.of(5)))
-				.apply(new AllWindowFunction<Tuple2<Integer, String>, String, GlobalWindow>() {
-					@Override
-					public void apply(GlobalWindow window,
-							Iterable<Tuple2<Integer, String>> values,
-							Collector<String> out) throws Exception {
-
-					}
-				});
-
-		assertEquals(TypeExtractor.getForClass(String.class), window.getType());
-
-		DataStream<CustomPOJO> flatten = window
-				.windowAll(GlobalWindows.create())
-				.trigger(PurgingTrigger.of(CountTrigger.of(5)))
-				.fold(new CustomPOJO(), new FoldFunction<String, CustomPOJO>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public CustomPOJO fold(CustomPOJO accumulator, String value) throws Exception {
-						return null;
-					}
-				});
-
-		assertEquals(TypeExtractor.getForClass(CustomPOJO.class), flatten.getType());
-	}
-
-	@Test
-	public void operatorTest() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Long> src = env.generateSequence(0, 0);
-
-		MapFunction<Long, Integer> mapFunction = new MapFunction<Long, Integer>() {
-			@Override
-			public Integer map(Long value) throws Exception {
-				return null;
-			}
-		};
-		DataStream<Integer> map = src.map(mapFunction);
-		map.addSink(new NoOpSink<Integer>());
-		assertEquals(mapFunction, getFunctionForDataStream(map));
-
-
-		FlatMapFunction<Long, Integer> flatMapFunction = new FlatMapFunction<Long, Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void flatMap(Long value, Collector<Integer> out) throws Exception {
-			}
-		};
-		DataStream<Integer> flatMap = src.flatMap(flatMapFunction);
-		flatMap.addSink(new NoOpSink<Integer>());
-		assertEquals(flatMapFunction, getFunctionForDataStream(flatMap));
-
-		FilterFunction<Integer> filterFunction = new FilterFunction<Integer>() {
-			@Override
-			public boolean filter(Integer value) throws Exception {
-				return false;
-			}
-		};
-
-		DataStream<Integer> unionFilter = map.union(flatMap)
-				.filter(filterFunction);
-
-		unionFilter.addSink(new NoOpSink<Integer>());
-
-		assertEquals(filterFunction, getFunctionForDataStream(unionFilter));
-
-		try {
-			env.getStreamGraph().getStreamEdge(map.getId(), unionFilter.getId());
-		} catch (RuntimeException e) {
-			fail(e.getMessage());
-		}
-
-		try {
-			env.getStreamGraph().getStreamEdge(flatMap.getId(), unionFilter.getId());
-		} catch (RuntimeException e) {
-			fail(e.getMessage());
-		}
-
-		OutputSelector<Integer> outputSelector = new OutputSelector<Integer>() {
-			@Override
-			public Iterable<String> select(Integer value) {
-				return null;
-			}
-		};
-
-		SplitStream<Integer> split = unionFilter.split(outputSelector);
-		split.select("dummy").addSink(new NoOpSink<Integer>());
-		List<OutputSelector<?>> outputSelectors = env.getStreamGraph().getStreamNode(unionFilter.getId()).getOutputSelectors();
-		assertEquals(1, outputSelectors.size());
-		assertEquals(outputSelector, outputSelectors.get(0));
-
-		DataStream<Integer> select = split.select("a");
-		DataStreamSink<Integer> sink = select.print();
-
-		StreamEdge splitEdge = env.getStreamGraph().getStreamEdge(unionFilter.getId(), sink.getTransformation().getId());
-		assertEquals("a", splitEdge.getSelectedNames().get(0));
-
-		ConnectedStreams<Integer, Integer> connect = map.connect(flatMap);
-		CoMapFunction<Integer, Integer, String> coMapper = new CoMapFunction<Integer, Integer, String>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public String map1(Integer value) {
-				return null;
-			}
-
-			@Override
-			public String map2(Integer value) {
-				return null;
-			}
-		};
-		DataStream<String> coMap = connect.map(coMapper);
-		coMap.addSink(new NoOpSink<String>());
-		assertEquals(coMapper, getFunctionForDataStream(coMap));
-
-		try {
-			env.getStreamGraph().getStreamEdge(map.getId(), coMap.getId());
-		} catch (RuntimeException e) {
-			fail(e.getMessage());
-		}
-
-		try {
-			env.getStreamGraph().getStreamEdge(flatMap.getId(), coMap.getId());
-		} catch (RuntimeException e) {
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void sinkKeyTest() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSink<Long> sink = env.generateSequence(1, 100).print();
-		assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getStatePartitioner() == null);
-		assertTrue(env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof ForwardPartitioner);
-
-		KeySelector<Long, Long> key1 = new KeySelector<Long, Long>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Long getKey(Long value) throws Exception {
-				return (long) 0;
-			}
-		};
-
-		DataStreamSink<Long> sink2 = env.generateSequence(1, 100).keyBy(key1).print();
-
-		assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner());
-		assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer());
-		assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer());
-		assertEquals(key1, env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner());
-		assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof HashPartitioner);
-
-		KeySelector<Long, Long> key2 = new KeySelector<Long, Long>() {
-
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Long getKey(Long value) throws Exception {
-				return (long) 0;
-			}
-		};
-
-		DataStreamSink<Long> sink3 = env.generateSequence(1, 100).keyBy(key2).print();
-
-		assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner() != null);
-		assertEquals(key2, env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getStatePartitioner());
-		assertTrue(env.getStreamGraph().getStreamNode(sink3.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof HashPartitioner);
-	}
-
-	@Test
-	public void testChannelSelectors() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStreamSource<Long> src = env.generateSequence(0, 0);
-
-		DataStream<Long> broadcast = src.broadcast();
-		DataStreamSink<Long> broadcastSink = broadcast.print();
-		StreamPartitioner<?> broadcastPartitioner =
-				env.getStreamGraph().getStreamEdge(src.getId(),
-						broadcastSink.getTransformation().getId()).getPartitioner();
-		assertTrue(broadcastPartitioner instanceof BroadcastPartitioner);
-
-		DataStream<Long> shuffle = src.shuffle();
-		DataStreamSink<Long> shuffleSink = shuffle.print();
-		StreamPartitioner<?> shufflePartitioner =
-				env.getStreamGraph().getStreamEdge(src.getId(),
-						shuffleSink.getTransformation().getId()).getPartitioner();
-		assertTrue(shufflePartitioner instanceof ShufflePartitioner);
-
-		DataStream<Long> forward = src.forward();
-		DataStreamSink<Long> forwardSink = forward.print();
-		StreamPartitioner<?> forwardPartitioner =
-				env.getStreamGraph().getStreamEdge(src.getId(),
-						forwardSink.getTransformation().getId()).getPartitioner();
-		assertTrue(forwardPartitioner instanceof ForwardPartitioner);
-
-		DataStream<Long> rebalance = src.rebalance();
-		DataStreamSink<Long> rebalanceSink = rebalance.print();
-		StreamPartitioner<?> rebalancePartitioner =
-				env.getStreamGraph().getStreamEdge(src.getId(),
-						rebalanceSink.getTransformation().getId()).getPartitioner();
-		assertTrue(rebalancePartitioner instanceof RebalancePartitioner);
-
-		DataStream<Long> global = src.global();
-		DataStreamSink<Long> globalSink = global.print();
-		StreamPartitioner<?> globalPartitioner =
-				env.getStreamGraph().getStreamEdge(src.getId(),
-						globalSink.getTransformation().getId()).getPartitioner();
-		assertTrue(globalPartitioner instanceof GlobalPartitioner);
-	}
-
-	/////////////////////////////////////////////////////////////
-	// Utilities
-	/////////////////////////////////////////////////////////////
-
-	private static StreamOperator<?> getOperatorForDataStream(DataStream<?> dataStream) {
-		StreamExecutionEnvironment env = dataStream.getExecutionEnvironment();
-		StreamGraph streamGraph = env.getStreamGraph();
-		return streamGraph.getStreamNode(dataStream.getId()).getOperator();
-	}
-
-	private static Function getFunctionForDataStream(DataStream<?> dataStream) {
-		AbstractUdfStreamOperator<?, ?> operator =
-				(AbstractUdfStreamOperator<?, ?>) getOperatorForDataStream(dataStream);
-		return operator.getUserFunction();
-	}
-
-	private static Integer createDownStreamId(DataStream<?> dataStream) {
-		return dataStream.print().getTransformation().getId();
-	}
-
-	private static boolean isKeyed(DataStream<?> dataStream) {
-		return dataStream instanceof KeyedStream;
-	}
-
-	@SuppressWarnings("rawtypes,unchecked")
-	private static Integer createDownStreamId(ConnectedStreams dataStream) {
-		SingleOutputStreamOperator<?, ?> coMap = dataStream.map(new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public Object map1(Tuple2<Long, Long> value) {
-				return null;
-			}
-
-			@Override
-			public Object map2(Tuple2<Long, Long> value) {
-				return null;
-			}
-		});
-		coMap.addSink(new NoOpSink());
-		return coMap.getId();
-	}
-
-	private static boolean isKeyed(ConnectedStreams<?, ?> dataStream) {
-		return (dataStream.getFirstInput() instanceof KeyedStream && dataStream.getSecondInput() instanceof KeyedStream);
-	}
-
-	private static boolean isPartitioned(StreamEdge edge) {
-		return edge.getPartitioner() instanceof HashPartitioner;
-	}
-
-	private static boolean isCustomPartitioned(StreamEdge edge) {
-		return edge.getPartitioner() instanceof CustomPartitionerWrapper;
-	}
-
-	private static class FirstSelector implements KeySelector<Tuple2<Long, Long>, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Long getKey(Tuple2<Long, Long> value) throws Exception {
-			return value.f0;
-		}
-	}
-
-	public static class CustomPOJO {
-		private String s;
-		private int i;
-
-		public CustomPOJO() {
-		}
-
-		public void setS(String s) {
-			this.s = s;
-		}
-
-		public void setI(int i) {
-			this.i = i;
-		}
-
-		public String getS() {
-			return s;
-		}
-
-		public int getI() {
-			return i;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
deleted file mode 100644
index bd97e84..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/IterateTest.java
+++ /dev/null
@@ -1,614 +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;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.flink.api.common.InvalidProgramException;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobVertex;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.datastream.IterativeStream;
-import org.apache.flink.streaming.api.datastream.IterativeStream.ConnectedIterativeStreams;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.graph.StreamNode;
-import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
-import org.apache.flink.streaming.util.EvenOddOutputSelector;
-import org.apache.flink.streaming.util.NoOpIntMap;
-import org.apache.flink.streaming.util.ReceiveCheckNoOpSink;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-@SuppressWarnings({ "unchecked", "unused", "serial" })
-public class IterateTest extends StreamingMultipleProgramsTestBase {
-
-	private static boolean iterated[];
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testIncorrectParallelism() throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source = env.fromElements(1, 10);
-
-		IterativeStream<Integer> iter1 = source.iterate();
-		SingleOutputStreamOperator<Integer, ?> map1 = iter1.map(NoOpIntMap);
-		iter1.closeWith(map1).print();
-	}
-
-	@Test
-	public void testDoubleClosing() throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
-
-		IterativeStream<Integer> iter1 = source.iterate();
-
-		iter1.closeWith(iter1.map(NoOpIntMap));
-		iter1.closeWith(iter1.map(NoOpIntMap));
-	}
-
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDifferingParallelism() throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10)
-				.map(NoOpIntMap);
-
-		IterativeStream<Integer> iter1 = source.iterate();
-
-
-		iter1.closeWith(iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2));
-
-	}
-
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testCoDifferingParallelism() throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
-
-		ConnectedIterativeStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
-				Integer.class);
-
-
-		coIter.closeWith(coIter.map(NoOpIntCoMap).setParallelism(DEFAULT_PARALLELISM / 2));
-
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testClosingFromOutOfLoop() throws Exception {
-
-		// this test verifies that we cannot close an iteration with a DataStream that does not
-		// have the iteration in its predecessors
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
-
-		IterativeStream<Integer> iter1 = source.iterate();
-		IterativeStream<Integer> iter2 = source.iterate();
-
-
-		iter2.closeWith(iter1.map(NoOpIntMap));
-
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testCoIterClosingFromOutOfLoop() throws Exception {
-
-		// this test verifies that we cannot close an iteration with a DataStream that does not
-		// have the iteration in its predecessors
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// introduce dummy mapper to get to correct parallelism
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
-
-		IterativeStream<Integer> iter1 = source.iterate();
-		ConnectedIterativeStreams<Integer, Integer> coIter = source.iterate().withFeedbackType(
-				Integer.class);
-
-
-		coIter.closeWith(iter1.map(NoOpIntMap));
-
-	}
-
-	@Test(expected = IllegalStateException.class)
-	public void testExecutionWithEmptyIteration() throws Exception {
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap);
-
-		IterativeStream<Integer> iter1 = source.iterate();
-
-		iter1.map(NoOpIntMap).print();
-
-		env.execute();
-	}
-
-	@Test
-	public void testImmutabilityWithCoiteration() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source = env.fromElements(1, 10).map(NoOpIntMap); // for rebalance
-
-		IterativeStream<Integer> iter1 = source.iterate();
-		// Calling withFeedbackType should create a new iteration
-		ConnectedIterativeStreams<Integer, String> iter2 = iter1.withFeedbackType(String.class);
-
-		iter1.closeWith(iter1.map(NoOpIntMap)).print();
-		iter2.closeWith(iter2.map(NoOpCoMap)).print();
-
-		StreamGraph graph = env.getStreamGraph();
-
-		assertEquals(2, graph.getIterationSourceSinkPairs().size());
-
-		for (Tuple2<StreamNode, StreamNode> sourceSinkPair: graph.getIterationSourceSinkPairs()) {
-			assertEquals(sourceSinkPair.f0.getOutEdges().get(0).getTargetVertex(), sourceSinkPair.f1.getInEdges().get(0).getSourceVertex());
-		}
-	}
-
-	@Test
-	public void testmultipleHeadsTailsSimple() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5)
-				.shuffle()
-				.map(NoOpIntMap).name("ParallelizeMapShuffle");
-		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap).name("ParallelizeMapRebalance");
-
-		IterativeStream<Integer> iter1 = source1.union(source2).iterate();
-
-		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("IterRebalanceMap").setParallelism(DEFAULT_PARALLELISM / 2);
-		DataStream<Integer> head2 = iter1.map(NoOpIntMap).name("IterForwardMap");
-		DataStreamSink<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).addSink(new ReceiveCheckNoOpSink<Integer>());
-		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
-
-		SplitStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap).name("EvenOddSourceMap")
-				.split(new EvenOddOutputSelector());
-
-		iter1.closeWith(source3.select("even").union(
-				head1.rebalance().map(NoOpIntMap).broadcast(), head2.shuffle()));
-
-		StreamGraph graph = env.getStreamGraph();
-
-		JobGraph jg = graph.getJobGraph();
-
-		assertEquals(1, graph.getIterationSourceSinkPairs().size());
-
-		Tuple2<StreamNode, StreamNode> sourceSinkPair = graph.getIterationSourceSinkPairs().iterator().next();
-		StreamNode itSource = sourceSinkPair.f0;
-		StreamNode itSink = sourceSinkPair.f1;
-
-		assertEquals(4, itSource.getOutEdges().size());
-		assertEquals(3, itSink.getInEdges().size());
-
-		assertEquals(itSource.getParallelism(), itSink.getParallelism());
-
-		for (StreamEdge edge : itSource.getOutEdges()) {
-			if (edge.getTargetVertex().getOperatorName().equals("IterRebalanceMap")) {
-				assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
-			} else if (edge.getTargetVertex().getOperatorName().equals("IterForwardMap")) {
-				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
-			}
-		}
-		for (StreamEdge edge : itSink.getInEdges()) {
-			if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapShuffle")) {
-				assertTrue(edge.getPartitioner() instanceof ShufflePartitioner);
-			}
-
-			if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("ParallelizeMapForward")) {
-				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
-			}
-
-			if (graph.getStreamNode(edge.getSourceId()).getOperatorName().equals("EvenOddSourceMap")) {
-				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
-				assertTrue(edge.getSelectedNames().contains("even"));
-			}
-		}
-
-		// Test co-location
-
-		JobVertex itSource1 = null;
-		JobVertex itSink1 = null;
-
-		for (JobVertex vertex : jg.getVertices()) {
-			if (vertex.getName().contains("IterationSource")) {
-				itSource1 = vertex;
-			} else if (vertex.getName().contains("IterationSink")) {
-
-				itSink1 = vertex;
-
-			}
-		}
-
-		assertTrue(itSource1.getCoLocationGroup() != null);
-		assertEquals(itSource1.getCoLocationGroup(), itSink1.getCoLocationGroup());
-	}
-
-	@Test
-	public void testmultipleHeadsTailsWithTailPartitioning() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> source1 = env.fromElements(1, 2, 3, 4, 5)
-				.shuffle()
-				.map(NoOpIntMap);
-
-		DataStream<Integer> source2 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap);
-
-		IterativeStream<Integer> iter1 = source1.union(source2).iterate();
-
-		DataStream<Integer> head1 = iter1.map(NoOpIntMap).name("map1");
-		DataStream<Integer> head2 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2).rebalance().name(
-				"shuffle");
-		DataStreamSink<Integer> head3 = iter1.map(NoOpIntMap).setParallelism(DEFAULT_PARALLELISM / 2)
-				.addSink(new ReceiveCheckNoOpSink<Integer>());
-		DataStreamSink<Integer> head4 = iter1.map(NoOpIntMap).addSink(new ReceiveCheckNoOpSink<Integer>());
-
-		SplitStream<Integer> source3 = env.fromElements(1, 2, 3, 4, 5)
-				.map(NoOpIntMap)
-				.name("split")
-				.split(new EvenOddOutputSelector());
-
-		iter1.closeWith(
-				source3.select("even").union(
-						head1.map(NoOpIntMap).broadcast().name("bc"),
-						head2.map(NoOpIntMap).shuffle()));
-
-		StreamGraph graph = env.getStreamGraph();
-
-		JobGraph jg = graph.getJobGraph();
-
-		assertEquals(1, graph.getIterationSourceSinkPairs().size());
-
-		Tuple2<StreamNode, StreamNode> sourceSinkPair = graph.getIterationSourceSinkPairs().iterator().next();
-		StreamNode itSource = sourceSinkPair.f0;
-		StreamNode itSink = sourceSinkPair.f1;
-
-		assertEquals(4, itSource.getOutEdges().size());
-		assertEquals(3, itSink.getInEdges().size());
-
-
-		assertEquals(itSource.getParallelism(), itSink.getParallelism());
-
-		for (StreamEdge edge : itSource.getOutEdges()) {
-			if (edge.getTargetVertex().getOperatorName().equals("map1")) {
-				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
-				assertEquals(4, edge.getTargetVertex().getParallelism());
-			} else if (edge.getTargetVertex().getOperatorName().equals("shuffle")) {
-				assertTrue(edge.getPartitioner() instanceof RebalancePartitioner);
-				assertEquals(2, edge.getTargetVertex().getParallelism());
-			}
-		}
-		for (StreamEdge edge : itSink.getInEdges()) {
-			String tailName = edge.getSourceVertex().getOperatorName();
-			if (tailName.equals("split")) {
-				assertTrue(edge.getPartitioner() instanceof ForwardPartitioner);
-				assertTrue(edge.getSelectedNames().contains("even"));
-			} else if (tailName.equals("bc")) {
-				assertTrue(edge.getPartitioner() instanceof BroadcastPartitioner);
-			} else if (tailName.equals("shuffle")) {
-				assertTrue(edge.getPartitioner() instanceof ShufflePartitioner);
-			}
-		}
-
-		// Test co-location
-
-		JobVertex itSource1 = null;
-		JobVertex itSink1 = null;
-
-		for (JobVertex vertex : jg.getVertices()) {
-			if (vertex.getName().contains("IterationSource")) {
-				itSource1 = vertex;
-			} else if (vertex.getName().contains("IterationSink")) {
-				itSink1 = vertex;
-			}
-		}
-
-		assertTrue(itSource1.getCoLocationGroup() != null);
-		assertTrue(itSink1.getCoLocationGroup() != null);
-
-		assertEquals(itSource1.getCoLocationGroup(), itSink1.getCoLocationGroup());
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testSimpleIteration() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		iterated = new boolean[DEFAULT_PARALLELISM];
-
-		DataStream<Boolean> source = env.fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false))
-				.map(NoOpBoolMap).name("ParallelizeMap");
-
-		IterativeStream<Boolean> iteration = source.iterate(3000);
-
-		DataStream<Boolean> increment = iteration.flatMap(new IterationHead()).map(NoOpBoolMap);
-
-		iteration.map(NoOpBoolMap).addSink(new ReceiveCheckNoOpSink());
-
-		iteration.closeWith(increment).addSink(new ReceiveCheckNoOpSink());
-
-		env.execute();
-
-		for (boolean iter : iterated) {
-			assertTrue(iter);
-		}
-
-	}
-
-	@Test
-	public void testCoIteration() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(2);
-
-		DataStream<String> otherSource = env.fromElements("1000", "2000")
-				.map(NoOpStrMap).name("ParallelizeMap");
-
-
-		ConnectedIterativeStreams<Integer, String> coIt = env.fromElements(0, 0)
-				.map(NoOpIntMap).name("ParallelizeMap")
-				.iterate(2000)
-				.withFeedbackType("String");
-
-		try {
-			coIt.keyBy(1, 2);
-			fail();
-		} catch (InvalidProgramException e) {
-			// this is expected
-		}
-
-		DataStream<String> head = coIt
-				.flatMap(new RichCoFlatMapFunction<Integer, String, String>() {
-
-					private static final long serialVersionUID = 1L;
-					boolean seenFromSource = false;
-
-					@Override
-					public void flatMap1(Integer value, Collector<String> out) throws Exception {
-						out.collect(((Integer) (value + 1)).toString());
-					}
-
-					@Override
-					public void flatMap2(String value, Collector<String> out) throws Exception {
-						Integer intVal = Integer.valueOf(value);
-						if (intVal < 2) {
-							out.collect(((Integer) (intVal + 1)).toString());
-						}
-						if (intVal == 1000 || intVal == 2000) {
-							seenFromSource = true;
-						}
-					}
-
-					@Override
-					public void close() {
-						assertTrue(seenFromSource);
-					}
-				});
-
-		coIt.map(new CoMapFunction<Integer, String, String>() {
-
-			@Override
-			public String map1(Integer value) throws Exception {
-				return value.toString();
-			}
-
-			@Override
-			public String map2(String value) throws Exception {
-				return value;
-			}
-		}).addSink(new ReceiveCheckNoOpSink<String>());
-
-		coIt.closeWith(head.broadcast().union(otherSource));
-
-		head.addSink(new TestSink()).setParallelism(1);
-
-		assertEquals(1, env.getStreamGraph().getIterationSourceSinkPairs().size());
-
-		env.execute();
-
-		Collections.sort(TestSink.collected);
-		assertEquals(Arrays.asList("1", "1", "2", "2", "2", "2"), TestSink.collected);
-	}
-
-	@Test
-	public void testGroupByFeedback() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(DEFAULT_PARALLELISM - 1);
-
-		KeySelector<Integer, Integer> key = new KeySelector<Integer, Integer>() {
-
-			@Override
-			public Integer getKey(Integer value) throws Exception {
-				return value % 3;
-			}
-		};
-
-		DataStream<Integer> source = env.fromElements(1, 2, 3)
-				.map(NoOpIntMap).name("ParallelizeMap");
-
-		IterativeStream<Integer> it = source.keyBy(key).iterate(3000);
-
-		DataStream<Integer> head = it.flatMap(new RichFlatMapFunction<Integer, Integer>() {
-
-			int received = 0;
-			int key = -1;
-
-			@Override
-			public void flatMap(Integer value, Collector<Integer> out) throws Exception {
-				received++;
-				if (key == -1) {
-					key = value % 3;
-				} else {
-					assertEquals(key, value % 3);
-				}
-				if (value > 0) {
-					out.collect(value - 1);
-				}
-			}
-
-			@Override
-			public void close() {
-				assertTrue(received > 1);
-			}
-		});
-
-		it.closeWith(head.keyBy(key).union(head.map(NoOpIntMap).keyBy(key))).addSink(new ReceiveCheckNoOpSink<Integer>());
-
-		env.execute();
-	}
-
-	@SuppressWarnings("deprecation")
-	@Test
-	public void testWithCheckPointing() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		env.enableCheckpointing();
-
-		DataStream<Boolean> source = env .fromCollection(Collections.nCopies(DEFAULT_PARALLELISM * 2, false))
-				.map(NoOpBoolMap).name("ParallelizeMap");
-
-
-		IterativeStream<Boolean> iteration = source.iterate(3000);
-
-		iteration.closeWith(iteration.flatMap(new IterationHead())).addSink(new ReceiveCheckNoOpSink<Boolean>());
-
-		try {
-			env.execute();
-
-			// this statement should never be reached
-			fail();
-		} catch (UnsupportedOperationException e) {
-			// expected behaviour
-		}
-
-		// Test force checkpointing
-
-		try {
-			env.enableCheckpointing(1, CheckpointingMode.EXACTLY_ONCE, false);
-			env.execute();
-
-			// this statement should never be reached
-			fail();
-		} catch (UnsupportedOperationException e) {
-			// expected behaviour
-		}
-
-		env.enableCheckpointing(1, CheckpointingMode.EXACTLY_ONCE, true);
-		env.getStreamGraph().getJobGraph();
-	}
-
-	public static final class IterationHead extends RichFlatMapFunction<Boolean, Boolean> {
-		public void flatMap(Boolean value, Collector<Boolean> out) throws Exception {
-			int indx = getRuntimeContext().getIndexOfThisSubtask();
-			if (value) {
-				iterated[indx] = true;
-			} else {
-				out.collect(true);
-			}
-		}
-	}
-
-	public static CoMapFunction<Integer, String, String> NoOpCoMap = new CoMapFunction<Integer, String, String>() {
-
-		public String map1(Integer value) throws Exception {
-			return value.toString();
-		}
-
-		public String map2(String value) throws Exception {
-			return value;
-		}
-	};
-
-	public static MapFunction<Integer, Integer> NoOpIntMap = new NoOpIntMap();
-
-	public static MapFunction<String, String> NoOpStrMap = new MapFunction<String, String>() {
-
-		public String map(String value) throws Exception {
-			return value;
-		}
-
-	};
-
-	public static CoMapFunction<Integer, Integer, Integer> NoOpIntCoMap = new CoMapFunction<Integer, Integer, Integer>() {
-
-		public Integer map1(Integer value) throws Exception {
-			return value;
-		}
-
-		public Integer map2(Integer value) throws Exception {
-			return value;
-		}
-
-	};
-
-	public static MapFunction<Boolean, Boolean> NoOpBoolMap = new MapFunction<Boolean, Boolean>() {
-
-		public Boolean map(Boolean value) throws Exception {
-			return value;
-		}
-
-	};
-
-	public static class TestSink implements SinkFunction<String> {
-
-		private static final long serialVersionUID = 1L;
-		public static List<String> collected = new ArrayList<String>();
-
-		@Override
-		public void invoke(String value) throws Exception {
-			collected.add(value);
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java
deleted file mode 100644
index 8525d37..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/OutputSplitterTest.java
+++ /dev/null
@@ -1,144 +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;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.junit.Test;
-
-public class OutputSplitterTest extends StreamingMultipleProgramsTestBase {
-
-	private static ArrayList<Integer> expectedSplitterResult = new ArrayList<Integer>();
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testOnMergedDataStream() throws Exception {
-		TestListResultSink<Integer> splitterResultSink1 = new TestListResultSink<Integer>();
-		TestListResultSink<Integer> splitterResultSink2 = new TestListResultSink<Integer>();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-		env.setBufferTimeout(1);
-
-		DataStream<Integer> d1 = env.fromElements(0, 2, 4, 6, 8);
-		DataStream<Integer> d2 = env.fromElements(1, 3, 5, 7, 9);
-
-		d1 = d1.union(d2);
-
-		d1.split(new OutputSelector<Integer>() {
-			private static final long serialVersionUID = 8354166915727490130L;
-
-			@Override
-			public Iterable<String> select(Integer value) {
-				List<String> s = new ArrayList<String>();
-				if (value > 4) {
-					s.add(">");
-				} else {
-					s.add("<");
-				}
-				return s;
-			}
-		}).select(">").addSink(splitterResultSink1);
-
-		d1.split(new OutputSelector<Integer>() {
-			private static final long serialVersionUID = -6822487543355994807L;
-
-			@Override
-			public Iterable<String> select(Integer value) {
-				List<String> s = new ArrayList<String>();
-				if (value % 3 == 0) {
-					s.add("yes");
-				} else {
-					s.add("no");
-				}
-				return s;
-			}
-		}).select("yes").addSink(splitterResultSink2);
-		env.execute();
-
-		expectedSplitterResult.clear();
-		expectedSplitterResult.addAll(Arrays.asList(5, 6, 7, 8, 9));
-		assertEquals(expectedSplitterResult, splitterResultSink1.getSortedResult());
-
-		expectedSplitterResult.clear();
-		expectedSplitterResult.addAll(Arrays.asList(0, 3, 6, 9));
-		assertEquals(expectedSplitterResult, splitterResultSink2.getSortedResult());
-	}
-
-	@Test
-	public void testOnSingleDataStream() throws Exception {
-		TestListResultSink<Integer> splitterResultSink1 = new TestListResultSink<Integer>();
-		TestListResultSink<Integer> splitterResultSink2 = new TestListResultSink<Integer>();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-		env.setBufferTimeout(1);
-
-		DataStream<Integer> ds = env.fromElements(0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
-
-		ds.split(new OutputSelector<Integer>() {
-			private static final long serialVersionUID = 2524335410904414121L;
-
-			@Override
-			public Iterable<String> select(Integer value) {
-				List<String> s = new ArrayList<String>();
-				if (value % 2 == 0) {
-					s.add("even");
-				} else {
-					s.add("odd");
-				}
-				return s;
-			}
-		}).select("even").addSink(splitterResultSink1);
-
-		ds.split(new OutputSelector<Integer>() {
-
-			private static final long serialVersionUID = -511693919586034092L;
-
-			@Override
-			public Iterable<String> select(Integer value) {
-				List<String> s = new ArrayList<String>();
-				if (value % 4 == 0) {
-					s.add("yes");
-				} else {
-					s.add("no");
-				}
-				return s;
-			}
-		}).select("yes").addSink(splitterResultSink2);
-		env.execute();
-
-		expectedSplitterResult.clear();
-		expectedSplitterResult.addAll(Arrays.asList(0, 2, 4, 6, 8));
-		assertEquals(expectedSplitterResult, splitterResultSink1.getSortedResult());
-
-		expectedSplitterResult.clear();
-		expectedSplitterResult.addAll(Arrays.asList(0, 4, 8));
-		assertEquals(expectedSplitterResult, splitterResultSink2.getSortedResult());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
deleted file mode 100644
index a6c6936..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
+++ /dev/null
@@ -1,268 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.Partitioner;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.NoOpIntMap;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestListResultSink;
-
-import org.junit.Test;
-
-/**
- * IT case that tests the different stream partitioning schemes.
- */
-public class PartitionerTest extends StreamingMultipleProgramsTestBase {
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testForwardFailsLowToHighParallelism() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		DataStream<Integer> src = env.fromElements(1, 2, 3);
-
-		// this doesn't work because it goes from 1 to 3
-		src.forward().map(new NoOpIntMap());
-
-		env.execute();
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testForwardFailsHightToLowParallelism() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// this does a rebalance that works
-		DataStream<Integer> src = env.fromElements(1, 2, 3).map(new NoOpIntMap());
-
-		// this doesn't work because it goes from 3 to 1
-		src.forward().map(new NoOpIntMap()).setParallelism(1);
-
-		env.execute();
-	}
-
-
-	@Test
-	public void partitionerTest() {
-
-		TestListResultSink<Tuple2<Integer, String>> hashPartitionResultSink =
-				new TestListResultSink<Tuple2<Integer, String>>();
-		TestListResultSink<Tuple2<Integer, String>> customPartitionResultSink =
-				new TestListResultSink<Tuple2<Integer, String>>();
-		TestListResultSink<Tuple2<Integer, String>> broadcastPartitionResultSink =
-				new TestListResultSink<Tuple2<Integer, String>>();
-		TestListResultSink<Tuple2<Integer, String>> forwardPartitionResultSink =
-				new TestListResultSink<Tuple2<Integer, String>>();
-		TestListResultSink<Tuple2<Integer, String>> rebalancePartitionResultSink =
-				new TestListResultSink<Tuple2<Integer, String>>();
-		TestListResultSink<Tuple2<Integer, String>> globalPartitionResultSink =
-				new TestListResultSink<Tuple2<Integer, String>>();
-
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		DataStream<Tuple1<String>> src = env.fromElements(
-				new Tuple1<String>("a"),
-				new Tuple1<String>("b"),
-				new Tuple1<String>("b"),
-				new Tuple1<String>("a"),
-				new Tuple1<String>("a"),
-				new Tuple1<String>("c"),
-				new Tuple1<String>("a")
-		);
-
-		// partition by hash
-		src
-				.partitionByHash(0)
-				.map(new SubtaskIndexAssigner())
-				.addSink(hashPartitionResultSink);
-
-		// partition custom
-		DataStream<Tuple2<Integer, String>> partitionCustom = src
-				.partitionCustom(new Partitioner<String>() {
-					@Override
-					public int partition(String key, int numPartitions) {
-						if (key.equals("c")) {
-							return 2;
-						} else {
-							return 0;
-						}
-					}
-				}, 0)
-				.map(new SubtaskIndexAssigner());
-
-		partitionCustom.addSink(customPartitionResultSink);
-
-		// partition broadcast
-		src.broadcast().map(new SubtaskIndexAssigner()).addSink(broadcastPartitionResultSink);
-
-		// partition rebalance
-		src.rebalance().map(new SubtaskIndexAssigner()).addSink(rebalancePartitionResultSink);
-
-		// partition forward
-		src.map(new MapFunction<Tuple1<String>, Tuple1<String>>() {
-			private static final long serialVersionUID = 1L;
-			@Override
-			public Tuple1<String> map(Tuple1<String> value) throws Exception {
-				return value;
-			}
-		})
-				.forward()
-				.map(new SubtaskIndexAssigner())
-				.addSink(forwardPartitionResultSink);
-
-		// partition global
-		src.global().map(new SubtaskIndexAssigner()).addSink(globalPartitionResultSink);
-
-		try {
-			env.execute();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-
-		List<Tuple2<Integer, String>> hashPartitionResult = hashPartitionResultSink.getResult();
-		List<Tuple2<Integer, String>> customPartitionResult = customPartitionResultSink.getResult();
-		List<Tuple2<Integer, String>> broadcastPartitionResult = broadcastPartitionResultSink.getResult();
-		List<Tuple2<Integer, String>> forwardPartitionResult = forwardPartitionResultSink.getResult();
-		List<Tuple2<Integer, String>> rebalancePartitionResult = rebalancePartitionResultSink.getResult();
-		List<Tuple2<Integer, String>> globalPartitionResult = globalPartitionResultSink.getResult();
-
-		verifyHashPartitioning(hashPartitionResult);
-		verifyCustomPartitioning(customPartitionResult);
-		verifyBroadcastPartitioning(broadcastPartitionResult);
-		verifyRebalancePartitioning(forwardPartitionResult);
-		verifyRebalancePartitioning(rebalancePartitionResult);
-		verifyGlobalPartitioning(globalPartitionResult);
-	}
-
-	private static void verifyHashPartitioning(List<Tuple2<Integer, String>> hashPartitionResult) {
-		HashMap<String, Integer> verifier = new HashMap<String, Integer>();
-		for (Tuple2<Integer, String> elem : hashPartitionResult) {
-			Integer subtaskIndex = verifier.get(elem.f1);
-			if (subtaskIndex == null) {
-				verifier.put(elem.f1, elem.f0);
-			} else if (subtaskIndex != elem.f0) {
-				fail();
-			}
-		}
-	}
-
-	private static void verifyCustomPartitioning(List<Tuple2<Integer, String>> customPartitionResult) {
-		for (Tuple2<Integer, String> stringWithSubtask : customPartitionResult) {
-			if (stringWithSubtask.f1.equals("c")) {
-				assertEquals(new Integer(2), stringWithSubtask.f0);
-			} else {
-				assertEquals(new Integer(0), stringWithSubtask.f0);
-			}
-		}
-	}
-
-	private static void verifyBroadcastPartitioning(List<Tuple2<Integer, String>> broadcastPartitionResult) {
-		List<Tuple2<Integer, String>> expected = Arrays.asList(
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(0, "b"),
-				new Tuple2<Integer, String>(0, "b"),
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(0, "c"),
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(1, "a"),
-				new Tuple2<Integer, String>(1, "b"),
-				new Tuple2<Integer, String>(1, "b"),
-				new Tuple2<Integer, String>(1, "a"),
-				new Tuple2<Integer, String>(1, "a"),
-				new Tuple2<Integer, String>(1, "c"),
-				new Tuple2<Integer, String>(1, "a"),
-				new Tuple2<Integer, String>(2, "a"),
-				new Tuple2<Integer, String>(2, "b"),
-				new Tuple2<Integer, String>(2, "b"),
-				new Tuple2<Integer, String>(2, "a"),
-				new Tuple2<Integer, String>(2, "a"),
-				new Tuple2<Integer, String>(2, "c"),
-				new Tuple2<Integer, String>(2, "a"));
-
-		assertEquals(
-				new HashSet<Tuple2<Integer, String>>(expected),
-				new HashSet<Tuple2<Integer, String>>(broadcastPartitionResult));
-	}
-
-	private static void verifyRebalancePartitioning(List<Tuple2<Integer, String>> rebalancePartitionResult) {
-		List<Tuple2<Integer, String>> expected = Arrays.asList(
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(1, "b"),
-				new Tuple2<Integer, String>(2, "b"),
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(1, "a"),
-				new Tuple2<Integer, String>(2, "c"),
-				new Tuple2<Integer, String>(0, "a"));
-
-		assertEquals(
-				new HashSet<Tuple2<Integer, String>>(expected),
-				new HashSet<Tuple2<Integer, String>>(rebalancePartitionResult));
-	}
-
-	private static void verifyGlobalPartitioning(List<Tuple2<Integer, String>> globalPartitionResult) {
-		List<Tuple2<Integer, String>> expected = Arrays.asList(
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(0, "b"),
-				new Tuple2<Integer, String>(0, "b"),
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(0, "a"),
-				new Tuple2<Integer, String>(0, "c"),
-				new Tuple2<Integer, String>(0, "a"));
-
-		assertEquals(
-				new HashSet<Tuple2<Integer, String>>(expected),
-				new HashSet<Tuple2<Integer, String>>(globalPartitionResult));
-	}
-
-	private static class SubtaskIndexAssigner extends RichMapFunction<Tuple1<String>, Tuple2<Integer, String>> {
-		private static final long serialVersionUID = 1L;
-
-		private int indexOfSubtask;
-
-		@Override
-		public void open(Configuration parameters) throws Exception {
-			super.open(parameters);
-			RuntimeContext runtimeContext = getRuntimeContext();
-			indexOfSubtask = runtimeContext.getIndexOfThisSubtask();
-		}
-
-		@Override
-		public Tuple2<Integer, String> map(Tuple1<String> value) throws Exception {
-			return new Tuple2<Integer, String>(indexOfSubtask, value.f0);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceFunctionTest.java
deleted file mode 100644
index b53649a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/SourceFunctionTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.core.testutils.CommonTestUtils;
-import org.apache.flink.streaming.api.functions.source.FromElementsFunction;
-import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
-import org.apache.flink.streaming.util.SourceFunctionUtil;
-import org.junit.Test;
-
-public class SourceFunctionTest {
-
-	@Test
-	public void fromElementsTest() throws Exception {
-		List<Integer> expectedList = Arrays.asList(1, 2, 3);
-		List<Integer> actualList = SourceFunctionUtil.runSourceFunction(CommonTestUtils.createCopySerializable(
-				new FromElementsFunction<Integer>(
-						IntSerializer.INSTANCE,
-						1,
-						2,
-						3)));
-		assertEquals(expectedList, actualList);
-	}
-
-	@Test
-	public void fromCollectionTest() throws Exception {
-		List<Integer> expectedList = Arrays.asList(1, 2, 3);
-		List<Integer> actualList = SourceFunctionUtil.runSourceFunction(
-				CommonTestUtils.createCopySerializable(new FromElementsFunction<Integer>(
-						IntSerializer.INSTANCE,
-						Arrays.asList(1, 2, 3))));
-		assertEquals(expectedList, actualList);
-	}
-
-	@Test
-	public void generateSequenceTest() throws Exception {
-		List<Long> expectedList = Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L);
-		List<Long> actualList = SourceFunctionUtil.runSourceFunction(new StatefulSequenceSource(1,
-				7));
-		assertEquals(expectedList, actualList);
-	}
-
-	@Test
-	public void socketTextStreamTest() throws Exception {
-		// TODO: does not work because we cannot set the internal socket anymore
-//		List<String> expectedList = Arrays.asList("a", "b", "c");
-//		List<String> actualList = new ArrayList<String>();
-//
-//		byte[] data = { 'a', '\n', 'b', '\n', 'c', '\n' };
-//
-//		Socket socket = mock(Socket.class);
-//		when(socket.getInputStream()).thenReturn(new ByteArrayInputStream(data));
-//		when(socket.isClosed()).thenReturn(false);
-//		when(socket.isConnected()).thenReturn(true);
-//
-//		SocketTextStreamFunction source = new SocketTextStreamFunction("", 0, '\n', 0);
-//		source.open(new Configuration());
-//		while (!source.reachedEnd()) {
-//			actualList.add(source.next());
-//		}
-//		assertEquals(expectedList, actualList);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
deleted file mode 100644
index 606259e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
+++ /dev/null
@@ -1,163 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.FromElementsFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.util.NoOpSink;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.util.SplittableIterator;
-import org.junit.Test;
-
-public class StreamExecutionEnvironmentTest extends StreamingMultipleProgramsTestBase {
-
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testFromCollectionParallelism() {
-		try {
-			TypeInformation<Integer> typeInfo = BasicTypeInfo.INT_TYPE_INFO;
-			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-			DataStreamSource<Integer> dataStream1 = env.fromCollection(new DummySplittableIterator<Integer>(), typeInfo);
-
-			try {
-				dataStream1.setParallelism(4);
-				fail("should throw an exception");
-			}
-			catch (IllegalArgumentException e) {
-				// expected
-			}
-
-			dataStream1.addSink(new NoOpSink<Integer>());
-	
-			DataStreamSource<Integer> dataStream2 = env.fromParallelCollection(new DummySplittableIterator<Integer>(),
-					typeInfo).setParallelism(4);
-
-			dataStream2.addSink(new NoOpSink<Integer>());
-
-			String plan = env.getExecutionPlan();
-
-			assertEquals("Parallelism of collection source must be 1.", 1, env.getStreamGraph().getStreamNode(dataStream1.getId()).getParallelism());
-			assertEquals("Parallelism of parallel collection source must be 4.",
-					4,
-					env.getStreamGraph().getStreamNode(dataStream2.getId()).getParallelism());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSources() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		SourceFunction<Integer> srcFun = new SourceFunction<Integer>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Integer> ctx) throws Exception {
-			}
-
-			@Override
-			public void cancel() {
-			}
-		};
-		DataStreamSource<Integer> src1 = env.addSource(srcFun);
-		src1.addSink(new NoOpSink<Integer>());
-		assertEquals(srcFun, getFunctionFromDataSource(src1));
-
-		List<Long> list = Arrays.asList(0L, 1L, 2L);
-
-		DataStreamSource<Long> src2 = env.generateSequence(0, 2);
-		assertTrue(getFunctionFromDataSource(src2) instanceof StatefulSequenceSource);
-
-		DataStreamSource<Long> src3 = env.fromElements(0L, 1L, 2L);
-		assertTrue(getFunctionFromDataSource(src3) instanceof FromElementsFunction);
-
-		DataStreamSource<Long> src4 = env.fromCollection(list);
-		assertTrue(getFunctionFromDataSource(src4) instanceof FromElementsFunction);
-	}
-
-	/////////////////////////////////////////////////////////////
-	// Utilities
-	/////////////////////////////////////////////////////////////
-
-
-	private static StreamOperator<?> getOperatorFromDataStream(DataStream<?> dataStream) {
-		StreamExecutionEnvironment env = dataStream.getExecutionEnvironment();
-		StreamGraph streamGraph = env.getStreamGraph();
-		return streamGraph.getStreamNode(dataStream.getId()).getOperator();
-	}
-
-	private static <T> SourceFunction<T> getFunctionFromDataSource(DataStreamSource<T> dataStreamSource) {
-		dataStreamSource.addSink(new NoOpSink<T>());
-		AbstractUdfStreamOperator<?, ?> operator =
-				(AbstractUdfStreamOperator<?, ?>) getOperatorFromDataStream(dataStreamSource);
-		return (SourceFunction<T>) operator.getUserFunction();
-	}
-
-	public static class DummySplittableIterator<T> extends SplittableIterator<T> {
-		private static final long serialVersionUID = 1312752876092210499L;
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public Iterator<T>[] split(int numPartitions) {
-			return (Iterator<T>[]) new Iterator<?>[0];
-		}
-
-		@Override
-		public int getMaximumNumberOfSplits() {
-			return 0;
-		}
-
-		@Override
-		public boolean hasNext() {
-			return false;
-		}
-
-		@Override
-		public T next() {
-			throw new NoSuchElementException();
-		}
-
-		@Override
-		public void remove() {
-			throw new UnsupportedOperationException();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
deleted file mode 100644
index 42febea..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/StreamingOperatorsITCase.java
+++ /dev/null
@@ -1,230 +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;
-
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class StreamingOperatorsITCase extends StreamingMultipleProgramsTestBase {
-
-	private String resultPath1;
-	private String resultPath2;
-	private String expected1;
-	private String expected2;
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws Exception {
-		resultPath1 = tempFolder.newFile().toURI().toString();
-		resultPath2 = tempFolder.newFile().toURI().toString();
-		expected1 = "";
-		expected2 = "";
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expected1, resultPath1);
-		compareResultsByLinesInMemory(expected2, resultPath2);
-	}
-
-	/**
-	 * Tests the proper functioning of the streaming fold operator. For this purpose, a stream
-	 * of Tuple2<Integer, Integer> is created. The stream is grouped according to the first tuple
-	 * value. Each group is folded where the second tuple value is summed up.
-	 *
-	 * @throws Exception
-	 */
-	@Test
-	public void testFoldOperation() throws Exception {
-		int numElements = 10;
-		int numKeys = 2;
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		DataStream<Tuple2<Integer, Integer>> sourceStream = env.addSource(new TupleSource(numElements, numKeys));
-
-		SplitStream<Tuple2<Integer, Integer>> splittedResult = sourceStream
-			.keyBy(0)
-			.fold(0, new FoldFunction<Tuple2<Integer, Integer>, Integer>() {
-				@Override
-				public Integer fold(Integer accumulator, Tuple2<Integer, Integer> value) throws Exception {
-					return accumulator + value.f1;
-				}
-			}).map(new RichMapFunction<Integer, Tuple2<Integer, Integer>>() {
-				@Override
-				public Tuple2<Integer, Integer> map(Integer value) throws Exception {
-					return new Tuple2<Integer, Integer>(getRuntimeContext().getIndexOfThisSubtask(), value);
-				}
-			}).split(new OutputSelector<Tuple2<Integer, Integer>>() {
-				@Override
-				public Iterable<String> select(Tuple2<Integer, Integer> value) {
-					List<String> output = new ArrayList<>();
-
-					output.add(value.f0 + "");
-
-					return output;
-				}
-			});
-
-		splittedResult.select("0").map(new MapFunction<Tuple2<Integer,Integer>, Integer>() {
-			@Override
-			public Integer map(Tuple2<Integer, Integer> value) throws Exception {
-				return value.f1;
-			}
-		}).writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-
-		splittedResult.select("1").map(new MapFunction<Tuple2<Integer, Integer>, Integer>() {
-			@Override
-			public Integer map(Tuple2<Integer, Integer> value) throws Exception {
-				return value.f1;
-			}
-		}).writeAsText(resultPath2, FileSystem.WriteMode.OVERWRITE);
-
-		StringBuilder builder1 = new StringBuilder();
-		StringBuilder builder2 = new StringBuilder();
-		int counter1 = 0;
-		int counter2 = 0;
-
-		for (int i = 0; i < numElements; i++) {
-			if (i % 2 == 0) {
-				counter1 += i;
-				builder1.append(counter1 + "\n");
-			} else {
-				counter2 += i;
-				builder2.append(counter2 + "\n");
-			}
-		}
-
-		expected1 = builder1.toString();
-		expected2 = builder2.toString();
-
-		env.execute();
-	}
-
-	/**
-	 * Tests whether the fold operation can also be called with non Java serializable types.
-	 */
-	@Test
-	public void testFoldOperationWithNonJavaSerializableType() throws Exception {
-		final int numElements = 10;
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		DataStream<Tuple2<Integer, NonSerializable>> input = env.addSource(new NonSerializableTupleSource(numElements));
-
-		input
-			.keyBy(0)
-			.fold(
-				new NonSerializable(42),
-				new FoldFunction<Tuple2<Integer, NonSerializable>, NonSerializable>() {
-					@Override
-					public NonSerializable fold(NonSerializable accumulator, Tuple2<Integer, NonSerializable> value) throws Exception {
-						return new NonSerializable(accumulator.value + value.f1.value);
-					}
-			})
-			.map(new MapFunction<NonSerializable, Integer>() {
-				@Override
-				public Integer map(NonSerializable value) throws Exception {
-					return value.value;
-				}
-			})
-			.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-
-		StringBuilder builder = new StringBuilder();
-
-		for (int i = 0; i < numElements; i++) {
-			builder.append(42 + i + "\n");
-		}
-
-		expected1 = builder.toString();
-
-		env.execute();
-	}
-
-	private static class NonSerializable {
-		// This makes the type non-serializable
-		private final Object obj = new Object();
-
-		private final int value;
-
-		public NonSerializable(int value) {
-			this.value = value;
-		}
-	}
-
-	private static class NonSerializableTupleSource implements SourceFunction<Tuple2<Integer, NonSerializable>> {
-		private final int numElements;
-
-		public NonSerializableTupleSource(int numElements) {
-			this.numElements = numElements;
-		}
-
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, NonSerializable>> ctx) throws Exception {
-			for (int i = 0; i < numElements; i++) {
-				ctx.collect(new Tuple2<Integer, NonSerializable>(i, new NonSerializable(i)));
-			}
-		}
-
-		@Override
-		public void cancel() {}
-	}
-
-	private static class TupleSource implements SourceFunction<Tuple2<Integer, Integer>> {
-
-		private final int numElements;
-		private final int numKeys;
-
-		public TupleSource(int numElements, int numKeys) {
-			this.numElements = numElements;
-			this.numKeys = numKeys;
-		}
-
-		@Override
-		public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-			for (int i = 0; i < numElements; i++) {
-				Tuple2<Integer, Integer> result = new Tuple2<>(i % numKeys, i);
-				ctx.collect(result);
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-}


[42/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
deleted file mode 100644
index d4a3a77..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
+++ /dev/null
@@ -1,499 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.functions.FoldFunction;
-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.aggregation.AggregationFunction;
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamGroupedFold;
-import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.transformations.PartitionTransformation;
-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.windowing.evictors.CountEvictor;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-import org.apache.flink.streaming.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;
-
-/**
- * A {@code KeyedStream} represents a {@link DataStream} on which operator state is
- * partitioned by key using a provided {@link KeySelector}. Typical operations supported by a
- * {@code DataStream} are also possible on a {@code KeyedStream}, with the exception of
- * partitioning methods such as shuffle, forward and keyBy.
- *
- * <p>
- * Reduce-style operations, such as {@link #reduce}, {@link #sum} and {@link #fold} work on elements
- * that have the same key.
- *
- * @param <T> The type of the elements in the Keyed Stream.
- * @param <KEY> The type of the key in the Keyed Stream.
- */
-public class KeyedStream<T, KEY> extends DataStream<T> {
-
-	/** The key selector that can get the key by which the stream if partitioned from the elements */
-	private final KeySelector<T, KEY> keySelector;
-
-	/** The type of the key by which the stream is partitioned */
-	private final TypeInformation<KEY> keyType;
-	
-	/**
-	 * Creates a new {@link KeyedStream} using the given {@link KeySelector}
-	 * to partition operator state by key.
-	 * 
-	 * @param dataStream
-	 *            Base stream of data
-	 * @param keySelector
-	 *            Function for determining state partitions
-	 */
-	public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
-		this(dataStream, keySelector, TypeExtractor.getKeySelectorTypes(keySelector, dataStream.getType()));
-	}
-
-	/**
-	 * Creates a new {@link KeyedStream} using the given {@link KeySelector}
-	 * to partition operator state by key.
-	 *
-	 * @param dataStream
-	 *            Base stream of data
-	 * @param keySelector
-	 *            Function for determining state partitions
-	 */
-	public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector, TypeInformation<KEY> keyType) {
-		super(dataStream.getExecutionEnvironment(), new PartitionTransformation<>(
-				dataStream.getTransformation(), new HashPartitioner<>(keySelector)));
-		this.keySelector = keySelector;
-		this.keyType = keyType;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  properties
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the key selector that can get the key by which the stream if partitioned from the elements.
-	 * @return The key selector for the key.
-	 */
-	public KeySelector<T, KEY> getKeySelector() {
-		return this.keySelector;
-	}
-
-	/**
-	 * Gets the type of the key by which the stream is partitioned. 
-	 * @return The type of the key by which the stream is partitioned.
-	 */
-	public TypeInformation<KEY> getKeyType() {
-		return keyType;
-	}
-
-	@Override
-	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
-		throw new UnsupportedOperationException("Cannot override partitioning for KeyedStream.");
-	}
-
-	// ------------------------------------------------------------------------
-	//  basic transformations
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
-			TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
-
-		SingleOutputStreamOperator<R, ?> returnStream = super.transform(operatorName, outTypeInfo,operator);
-
-		// inject the key selector and key type
-		OneInputTransformation<T, R> transform = (OneInputTransformation<T, R>) returnStream.getTransformation();
-		transform.setStateKeySelector(keySelector);
-		transform.setStateKeyType(keyType);
-		
-		return returnStream;
-	}
-	
-	@Override
-	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
-		DataStreamSink<T> result = super.addSink(sinkFunction);
-		result.getTransformation().setStateKeySelector(keySelector);
-		result.getTransformation().setStateKeyType(keyType);
-		return result;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Windowing
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Windows this {@code KeyedStream} 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 WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size) {
-		return window(TumblingTimeWindows.of(size));
-	}
-
-	/**
-	 * Windows this {@code KeyedStream} 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 WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size, AbstractTime slide) {
-		return window(SlidingTimeWindows.of(size, slide));
-	}
-
-	/**
-	 * Windows this {@code KeyedStream} into tumbling count windows.
-	 *
-	 * @param size The size of the windows in number of elements.
-	 */
-	public WindowedStream<T, KEY, GlobalWindow> countWindow(long size) {
-		return window(GlobalWindows.create()).trigger(PurgingTrigger.of(CountTrigger.of(size)));
-	}
-
-	/**
-	 * Windows this {@code KeyedStream} into sliding count windows.
-	 *
-	 * @param size The size of the windows in number of elements.
-	 * @param slide The slide interval in number of elements.
-	 */
-	public WindowedStream<T, KEY, GlobalWindow> countWindow(long size, long slide) {
-		return window(GlobalWindows.create())
-				.evictor(CountEvictor.of(size))
-				.trigger(CountTrigger.of(slide));
-	}
-
-	/**
-	 * Windows this data stream to a {@code WindowedStream}, 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.
-	 *
-	 * <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.
-	 *
-	 * @param assigner The {@code WindowAssigner} that assigns elements to windows.
-	 * @return The trigger windows data stream.
-	 */
-	public <W extends Window> WindowedStream<T, KEY, W> window(WindowAssigner<? super T, W> assigner) {
-		return new WindowedStream<>(this, assigner);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Non-Windowed aggregation operations
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Applies a reduce transformation on the grouped data stream grouped on by
-	 * the given key position. The {@link ReduceFunction} will receive input
-	 * values based on the key value. Only input values with the same key will
-	 * go to the same reducer.
-	 *
-	 * @param reducer
-	 *            The {@link ReduceFunction} that will be called for every
-	 *            element of the input values with the same key.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> reducer) {
-		return transform("Keyed Reduce", getType(), new StreamGroupedReduce<T>(
-				clean(reducer), getType().createSerializer(getExecutionConfig())));
-	}
-
-	/**
-	 * Applies a fold transformation on the grouped data stream grouped on by
-	 * the given key position. The {@link FoldFunction} will receive input
-	 * values based on the key value. Only input values with the same key will
-	 * go to the same folder.
-	 *
-	 * @param folder
-	 *            The {@link FoldFunction} that will be called for every element
-	 *            of the input values with the same key.
-	 * @param initialValue
-	 *            The initialValue passed to the folders for each key.
-	 * @return The transformed DataStream.
-	 */
-	public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> folder) {
-
-		TypeInformation<R> outType = TypeExtractor.getFoldReturnTypes(
-				clean(folder), getType(), Utils.getCallLocationName(), true);
-
-		return transform("Keyed Fold", outType, new StreamGroupedFold<>(clean(folder), initialValue));
-	}
-
-	/**
-	 * Applies an aggregation that gives a rolling sum of the data stream at the
-	 * given position grouped by the given key. An independent aggregate is kept
-	 * per key.
-	 *
-	 * @param positionToSum
-	 *            The position in the data point to sum
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(int positionToSum) {
-		return aggregate(new SumAggregator<>(positionToSum, getType(), getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current sum of the pojo data
-	 * stream at the given field expressionby the given key. An independent
-	 * aggregate is kept per key. A field expression is either the name of a
-	 * public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> sum(String field) {
-		return aggregate(new SumAggregator<>(field, getType(), getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current minimum of the data
-	 * stream at the given position by the given key. An independent aggregate
-	 * is kept per key.
-	 *
-	 * @param positionToMin
-	 *            The position in the data point to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(int positionToMin) {
-		return aggregate(new ComparableAggregator<>(positionToMin, getType(), AggregationFunction.AggregationType.MIN,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current minimum of the pojo
-	 * data stream at the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> min(String field) {
-		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MIN,
-				false, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that gives the current maximum of the data stream
-	 * at the given position by the given key. An independent aggregate is kept
-	 * per key.
-	 *
-	 * @param positionToMax
-	 *            The position in the data point to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(int positionToMax) {
-		return aggregate(new ComparableAggregator<>(positionToMax, getType(), AggregationFunction.AggregationType.MAX,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current maximum of the pojo
-	 * data stream at the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> max(String field) {
-		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAX,
-				false, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current minimum element of the
-	 * pojo data stream by the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @param first
-	 *            If True then in case of field equality the first object will
-	 *            be returned
-	 * @return The transformed DataStream.
-	 */
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	public SingleOutputStreamOperator<T, ?> minBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator(field, getType(), AggregationFunction.AggregationType.MINBY,
-				first, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current maximum element of the
-	 * pojo data stream by the given field expression by the given key. An
-	 * independent aggregate is kept per key. A field expression is either the
-	 * name of a public field or a getter method with parentheses of the
-	 * {@link DataStream}S underlying type. A dot can be used to drill down into
-	 * objects, as in {@code "field1.getInnerField2()" }.
-	 *
-	 * @param field
-	 *            The field expression based on which the aggregation will be
-	 *            applied.
-	 * @param first
-	 *            If True then in case of field equality the first object will
-	 *            be returned
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) {
-		return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAXBY,
-				first, getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * minimum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the minimum value at the
-	 * given position, the operator returns the first one by default.
-	 *
-	 * @param positionToMinBy
-	 *            The position in the data point to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * minimum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the minimum value at the
-	 * given position, the operator returns the first one by default.
-	 *
-	 * @param positionToMinBy
-	 *            The position in the data point to minimize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(String positionToMinBy) {
-		return this.minBy(positionToMinBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * minimum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the minimum value at the
-	 * given position, the operator returns either the first or last one,
-	 * depending on the parameter set.
-	 *
-	 * @param positionToMinBy
-	 *            The position in the data point to minimize
-	 * @param first
-	 *            If true, then the operator return the first element with the
-	 *            minimal value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) {
-		return aggregate(new ComparableAggregator<T>(positionToMinBy, getType(), AggregationFunction.AggregationType.MINBY, first,
-				getExecutionConfig()));
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * maximum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the maximum value at the
-	 * given position, the operator returns the first one by default.
-	 *
-	 * @param positionToMaxBy
-	 *            The position in the data point to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * maximum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the maximum value at the
-	 * given position, the operator returns the first one by default.
-	 *
-	 * @param positionToMaxBy
-	 *            The position in the data point to maximize
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(String positionToMaxBy) {
-		return this.maxBy(positionToMaxBy, true);
-	}
-
-	/**
-	 * Applies an aggregation that that gives the current element with the
-	 * maximum value at the given position by the given key. An independent
-	 * aggregate is kept per key. If more elements have the maximum value at the
-	 * given position, the operator returns either the first or last one,
-	 * depending on the parameter set.
-	 *
-	 * @param positionToMaxBy
-	 *            The position in the data point to maximize.
-	 * @param first
-	 *            If true, then the operator return the first element with the
-	 *            maximum value, otherwise returns the last
-	 * @return The transformed DataStream.
-	 */
-	public SingleOutputStreamOperator<T, ?> maxBy(int positionToMaxBy, boolean first) {
-		return aggregate(new ComparableAggregator<>(positionToMaxBy, getType(), AggregationFunction.AggregationType.MAXBY, first,
-				getExecutionConfig()));
-	}
-
-	protected SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregate) {
-		StreamGroupedReduce<T> operator = new StreamGroupedReduce<T>(
-				clean(aggregate), getType().createSerializer(getExecutionConfig()));
-		return transform("Keyed Aggregation", getType(), operator);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
deleted file mode 100644
index 33d5a3c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ /dev/null
@@ -1,309 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.TypeInfoParser;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy;
-import org.apache.flink.streaming.api.operators.ChainingStrategy;
-import org.apache.flink.streaming.api.transformations.PartitionTransformation;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-
-/**
- * The SingleOutputStreamOperator represents a user defined transformation
- * applied on a {@link DataStream} with one predefined output type.
- *
- * @param <T> The type of the elements in this Stream
- * @param <O> Type of the operator.
- */
-public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<T, O>> extends DataStream<T> {
-
-	protected SingleOutputStreamOperator(StreamExecutionEnvironment environment, StreamTransformation<T> transformation) {
-		super(environment, transformation);
-	}
-
-	/**
-	 * Gets the name of the current data stream. This name is
-	 * used by the visualization and logging during runtime.
-	 *
-	 * @return Name of the stream.
-	 */
-	public String getName() {
-		return transformation.getName();
-	}
-
-	/**
-	 * Sets the name of the current data stream. This name is
-	 * used by the visualization and logging during runtime.
-	 *
-	 * @return The named operator.
-	 */
-	public SingleOutputStreamOperator<T, O> name(String name){
-		transformation.setName(name);
-		return this;
-	}
-
-	/**
-	 * Sets the parallelism for this operator. The degree must be 1 or more.
-	 * 
-	 * @param parallelism
-	 *            The parallelism for this operator.
-	 * @return The operator with set parallelism.
-	 */
-	public SingleOutputStreamOperator<T, O> setParallelism(int parallelism) {
-		if (parallelism < 1) {
-			throw new IllegalArgumentException("The parallelism of an operator must be at least 1.");
-		}
-
-		transformation.setParallelism(parallelism);
-
-		return this;
-	}
-
-	/**
-	 * Sets the maximum time frequency (ms) for the flushing of the output
-	 * buffer. By default the output buffers flush only when they are full.
-	 * 
-	 * @param timeoutMillis
-	 *            The maximum time between two output flushes.
-	 * @return The operator with buffer timeout set.
-	 */
-	public SingleOutputStreamOperator<T, O> setBufferTimeout(long timeoutMillis) {
-		transformation.setBufferTimeout(timeoutMillis);
-		return this;
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<T, O> broadcast() {
-		return (SingleOutputStreamOperator<T, O>) super.broadcast();
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<T, O> shuffle() {
-		return (SingleOutputStreamOperator<T, O>) super.shuffle();
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<T, O> forward() {
-		return (SingleOutputStreamOperator<T, O>) super.forward();
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<T, O> rebalance() {
-		return (SingleOutputStreamOperator<T, O>) super.rebalance();
-	}
-
-	@SuppressWarnings("unchecked")
-	public SingleOutputStreamOperator<T, O> global() {
-		return (SingleOutputStreamOperator<T, O>) super.global();
-	}
-
-	/**
-	 * Sets the {@link ChainingStrategy} for the given operator affecting the
-	 * way operators will possibly be co-located on the same thread for
-	 * increased performance.
-	 * 
-	 * @param strategy
-	 *            The selected {@link ChainingStrategy}
-	 * @return The operator with the modified chaining strategy
-	 */
-	private SingleOutputStreamOperator<T, O> setChainingStrategy(ChainingStrategy strategy) {
-		this.transformation.setChainingStrategy(strategy);
-		return this;
-	}
-
-	/**
-	 * Turns off chaining for this operator so thread co-location will not be
-	 * used as an optimization. </p> Chaining can be turned off for the whole
-	 * job by {@link StreamExecutionEnvironment#disableOperatorChaining()}
-	 * however it is not advised for performance considerations.
-	 * 
-	 * @return The operator with chaining disabled
-	 */
-	public SingleOutputStreamOperator<T, O> disableChaining() {
-		return setChainingStrategy(ChainingStrategy.NEVER);
-	}
-
-	/**
-	 * Starts a new task chain beginning at this operator. This operator will
-	 * not be chained (thread co-located for increased performance) to any
-	 * previous tasks even if possible.
-	 * 
-	 * @return The operator with chaining set.
-	 */
-	public SingleOutputStreamOperator<T, O> startNewChain() {
-		return setChainingStrategy(ChainingStrategy.HEAD);
-	}
-
-	/**
-	 * Adds a type information hint about the return type of this operator. 
-	 * 
-	 * <p>
-	 * Type hints are important in cases where the Java compiler
-	 * throws away generic type information necessary for efficient execution.
-	 * 
-	 * <p>
-	 * This method takes a type information string that will be parsed. A type information string can contain the following
-	 * types:
-	 *
-	 * <ul>
-	 * <li>Basic types such as <code>Integer</code>, <code>String</code>, etc.
-	 * <li>Basic type arrays such as <code>Integer[]</code>,
-	 * <code>String[]</code>, etc.
-	 * <li>Tuple types such as <code>Tuple1&lt;TYPE0&gt;</code>,
-	 * <code>Tuple2&lt;TYPE0, TYPE1&gt;</code>, etc.</li>
-	 * <li>Pojo types such as <code>org.my.MyPojo&lt;myFieldName=TYPE0,myFieldName2=TYPE1&gt;</code>, etc.</li>
-	 * <li>Generic types such as <code>java.lang.Class</code>, etc.
-	 * <li>Custom type arrays such as <code>org.my.CustomClass[]</code>,
-	 * <code>org.my.CustomClass$StaticInnerClass[]</code>, etc.
-	 * <li>Value types such as <code>DoubleValue</code>,
-	 * <code>StringValue</code>, <code>IntegerValue</code>, etc.</li>
-	 * <li>Tuple array types such as <code>Tuple2&lt;TYPE0,TYPE1&gt;[], etc.</code></li>
-	 * <li>Writable types such as <code>Writable&lt;org.my.CustomWritable&gt;</code></li>
-	 * <li>Enum types such as <code>Enum&lt;org.my.CustomEnum&gt;</code></li>
-	 * </ul>
-	 *
-	 * Example:
-	 * <code>"Tuple2&lt;String,Tuple2&lt;Integer,org.my.MyJob$Pojo&lt;word=String&gt;&gt;&gt;"</code>
-	 *
-	 * @param typeInfoString
-	 *            type information string to be parsed
-	 * @return This operator with a given return type hint.
-	 */
-	public O returns(String typeInfoString) {
-		if (typeInfoString == null) {
-			throw new IllegalArgumentException("Type information string must not be null.");
-		}
-		return returns(TypeInfoParser.<T>parse(typeInfoString));
-	}
-	
-	/**
-	 * Adds a type information hint about the return type of this operator. 
-	 * 
-	 * <p>
-	 * Type hints are important in cases where the Java compiler
-	 * throws away generic type information necessary for efficient execution.
-	 * 
-	 * <p>
-	 * This method takes an instance of {@link org.apache.flink.api.common.typeinfo.TypeInformation} such as:
-	 * 
-	 * <ul>
-	 * <li>{@link org.apache.flink.api.common.typeinfo.BasicTypeInfo}</li>
-	 * <li>{@link org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo}</li>
-	 * <li>{@link org.apache.flink.api.java.typeutils.TupleTypeInfo}</li>
-	 * <li>{@link org.apache.flink.api.java.typeutils.PojoTypeInfo}</li>
-	 * <li>{@link org.apache.flink.api.java.typeutils.WritableTypeInfo}</li>
-	 * <li>{@link org.apache.flink.api.java.typeutils.ValueTypeInfo}</li>
-	 * <li>etc.</li>
-	 * </ul>
-	 *
-	 * @param typeInfo
-	 *            type information as a return type hint
-	 * @return This operator with a given return type hint.
-	 */
-	public O returns(TypeInformation<T> typeInfo) {
-		if (typeInfo == null) {
-			throw new IllegalArgumentException("Type information must not be null.");
-		}
-		transformation.setOutputType(typeInfo);
-		@SuppressWarnings("unchecked")
-		O returnType = (O) this;
-		return returnType;
-	}
-	
-	/**
-	 * Adds a type information hint about the return type of this operator. 
-	 * 
-	 * <p>
-	 * Type hints are important in cases where the Java compiler
-	 * throws away generic type information necessary for efficient execution.
-	 * 
-	 * <p>
-	 * This method takes a class that will be analyzed by Flink's type extraction capabilities.
-	 * 
-	 * <p>
-	 * Examples for classes are:
-	 * <ul>
-	 * <li>Basic types such as <code>Integer.class</code>, <code>String.class</code>, etc.</li>
-	 * <li>POJOs such as <code>MyPojo.class</code></li>
-	 * <li>Classes that <b>extend</b> tuples. Classes like <code>Tuple1.class</code>,<code>Tuple2.class</code>, etc. are <b>not</b> sufficient.</li>
-	 * <li>Arrays such as <code>String[].class</code>, etc.</li>
-	 * </ul>
-	 *
-	 * @param typeClass
-	 *            class as a return type hint
-	 * @return This operator with a given return type hint.
-	 */
-	@SuppressWarnings("unchecked")
-	public O returns(Class<T> typeClass) {
-		if (typeClass == null) {
-			throw new IllegalArgumentException("Type class must not be null.");
-		}
-		
-		try {
-			TypeInformation<T> ti = (TypeInformation<T>) TypeExtractor.createTypeInfo(typeClass);
-			return returns(ti);
-		}
-		catch (InvalidTypesException e) {
-			throw new InvalidTypesException("The given class is not suited for providing necessary type information.", e);
-		}
-	}
-
-	@Override
-	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
-		return new SingleOutputStreamOperator<T, O>(this.getExecutionEnvironment(), new PartitionTransformation<T>(this.getTransformation(), partitioner));
-	}
-
-	/**
-	 * By default all operators in a streaming job share the same resource
-	 * group. Each resource group takes as many task manager slots as the
-	 * maximum parallelism operator in that group. Task chaining is only
-	 * possible within one resource group. By calling this method, this
-	 * operators starts a new resource group and all subsequent operators will
-	 * be added to this group unless specified otherwise. </p> Please note that
-	 * local executions have by default as many available task slots as the
-	 * environment parallelism, so in order to start a new resource group the
-	 * degree of parallelism for the operators must be decreased from the
-	 * default.
-	 * 
-	 * @return The operator as a part of a new resource group.
-	 */
-	public SingleOutputStreamOperator<T, O> startNewResourceGroup() {
-		transformation.setResourceStrategy(ResourceStrategy.NEWGROUP);
-		return this;
-	}
-
-	/**
-	 * Isolates the operator in its own resource group. This will cause the
-	 * operator to grab as many task slots as its degree of parallelism. If
-	 * there are no free resources available, the job will fail to start. It
-	 * also disables chaining for this operator </p>All subsequent operators are
-	 * assigned to the default resource group.
-	 * 
-	 * @return The operator with isolated resource group.
-	 */
-	public SingleOutputStreamOperator<T, O> isolateResources() {
-		transformation.setResourceStrategy(ResourceStrategy.ISOLATE);
-		return this;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
deleted file mode 100644
index 11ee7f2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.transformations.SelectTransformation;
-import org.apache.flink.streaming.api.transformations.SplitTransformation;
-
-/**
- * The SplitStream represents an operator that has been split using an
- * {@link OutputSelector}. Named outputs can be selected using the
- * {@link #select} function. To apply transformation on the whole output simply
- * call the transformation on the SplitStream
- *
- * @param <OUT> The type of the elements in the Stream
- */
-public class SplitStream<OUT> extends DataStream<OUT> {
-
-	protected SplitStream(DataStream<OUT> dataStream, OutputSelector<OUT> outputSelector) {
-		super(dataStream.getExecutionEnvironment(), new SplitTransformation<OUT>(dataStream.getTransformation(), outputSelector));
-	}
-
-	/**
-	 * Sets the output names for which the next operator will receive values.
-	 * 
-	 * @param outputNames
-	 *            The output names for which the operator will receive the
-	 *            input.
-	 * @return Returns the selected DataStream
-	 */
-	public DataStream<OUT> select(String... outputNames) {
-		return selectOutput(outputNames);
-	}
-
-	private DataStream<OUT> selectOutput(String[] outputNames) {
-		for (String outName : outputNames) {
-			if (outName == null) {
-				throw new RuntimeException("Selected names must not be null");
-			}
-		}
-
-		SelectTransformation<OUT> selectTransform = new SelectTransformation<OUT>(this.getTransformation(), Lists.newArrayList(outputNames));
-		return new DataStream<OUT>(this.getExecutionEnvironment(), selectTransform);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
deleted file mode 100644
index 149d7a8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/StreamProjection.java
+++ /dev/null
@@ -1,484 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.datastream;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple10;
-import org.apache.flink.api.java.tuple.Tuple11;
-import org.apache.flink.api.java.tuple.Tuple12;
-import org.apache.flink.api.java.tuple.Tuple13;
-import org.apache.flink.api.java.tuple.Tuple14;
-import org.apache.flink.api.java.tuple.Tuple15;
-import org.apache.flink.api.java.tuple.Tuple16;
-import org.apache.flink.api.java.tuple.Tuple17;
-import org.apache.flink.api.java.tuple.Tuple18;
-import org.apache.flink.api.java.tuple.Tuple19;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple20;
-import org.apache.flink.api.java.tuple.Tuple21;
-import org.apache.flink.api.java.tuple.Tuple22;
-import org.apache.flink.api.java.tuple.Tuple23;
-import org.apache.flink.api.java.tuple.Tuple24;
-import org.apache.flink.api.java.tuple.Tuple25;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple6;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.java.tuple.Tuple8;
-import org.apache.flink.api.java.tuple.Tuple9;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.streaming.api.operators.StreamProject;
-
-import com.google.common.base.Preconditions;
-
-public class StreamProjection<IN> {
-
-	private DataStream<IN> dataStream;
-	private int[] fieldIndexes;
-
-	protected StreamProjection(DataStream<IN> dataStream, int[] fieldIndexes) {
-		if (!dataStream.getType().isTupleType()) {
-			throw new RuntimeException("Only Tuple DataStreams can be projected");
-		}
-		if(fieldIndexes.length == 0) {
-			throw new IllegalArgumentException("project() needs to select at least one (1) field.");
-		} else if(fieldIndexes.length > Tuple.MAX_ARITY - 1) {
-			throw new IllegalArgumentException(
-					"project() may select only up to (" + (Tuple.MAX_ARITY - 1) + ") fields.");
-		}
-
-		int maxFieldIndex = (dataStream.getType()).getArity();
-		for(int i = 0; i < fieldIndexes.length; i++) {
-			Preconditions.checkElementIndex(fieldIndexes[i], maxFieldIndex);
-		}
-
-		this.dataStream = dataStream;
-		this.fieldIndexes = fieldIndexes;
-	}
-
-	/**
-	 * Chooses a projectTupleX according to the length of
-	 * {@link org.apache.flink.streaming.api.datastream.StreamProjection#fieldIndexes}
-	 *
-	 * @return The projected DataStream.
-	 * @see org.apache.flink.api.java.operators.ProjectOperator.Projection
-	 */
-	@SuppressWarnings("unchecked")
-	public <OUT extends Tuple> SingleOutputStreamOperator<OUT, ?> projectTupleX() {
-		SingleOutputStreamOperator<OUT, ?> projOperator = null;
-
-		switch (fieldIndexes.length) {
-			case 1: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple1(); break;
-			case 2: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple2(); break;
-			case 3: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple3(); break;
-			case 4: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple4(); break;
-			case 5: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple5(); break;
-			case 6: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple6(); break;
-			case 7: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple7(); break;
-			case 8: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple8(); break;
-			case 9: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple9(); break;
-			case 10: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple10(); break;
-			case 11: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple11(); break;
-			case 12: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple12(); break;
-			case 13: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple13(); break;
-			case 14: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple14(); break;
-			case 15: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple15(); break;
-			case 16: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple16(); break;
-			case 17: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple17(); break;
-			case 18: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple18(); break;
-			case 19: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple19(); break;
-			case 20: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple20(); break;
-			case 21: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple21(); break;
-			case 22: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple22(); break;
-			case 23: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple23(); break;
-			case 24: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple24(); break;
-			case 25: projOperator = (SingleOutputStreamOperator<OUT, ?>) projectTuple25(); break;
-			default:
-				throw new IllegalStateException("Excessive arity in tuple.");
-		}
-
-		return projOperator;
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0> SingleOutputStreamOperator<Tuple1<T0>, ?> projectTuple1() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple1<T0>> tType = new TupleTypeInfo<Tuple1<T0>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple1<T0>>(
-				fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1> SingleOutputStreamOperator<Tuple2<T0, T1>, ?> projectTuple2() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple2<T0, T1>> tType = new TupleTypeInfo<Tuple2<T0, T1>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple2<T0, T1>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2> SingleOutputStreamOperator<Tuple3<T0, T1, T2>, ?> projectTuple3() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple3<T0, T1, T2>> tType = new TupleTypeInfo<Tuple3<T0, T1, T2>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple3<T0, T1, T2>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3> SingleOutputStreamOperator<Tuple4<T0, T1, T2, T3>, ?> projectTuple4() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple4<T0, T1, T2, T3>> tType = new TupleTypeInfo<Tuple4<T0, T1, T2, T3>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple4<T0, T1, T2, T3>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4> SingleOutputStreamOperator<Tuple5<T0, T1, T2, T3, T4>, ?> projectTuple5() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>> tType = new TupleTypeInfo<Tuple5<T0, T1, T2, T3, T4>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple5<T0, T1, T2, T3, T4>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5> SingleOutputStreamOperator<Tuple6<T0, T1, T2, T3, T4, T5>, ?> projectTuple6() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>> tType = new TupleTypeInfo<Tuple6<T0, T1, T2, T3, T4, T5>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple6<T0, T1, T2, T3, T4, T5>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6> SingleOutputStreamOperator<Tuple7<T0, T1, T2, T3, T4, T5, T6>, ?> projectTuple7() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>> tType = new TupleTypeInfo<Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple7<T0, T1, T2, T3, T4, T5, T6>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7> SingleOutputStreamOperator<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>, ?> projectTuple8() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>> tType = new TupleTypeInfo<Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple8<T0, T1, T2, T3, T4, T5, T6, T7>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8> SingleOutputStreamOperator<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>, ?> projectTuple9() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>> tType = new TupleTypeInfo<Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple9<T0, T1, T2, T3, T4, T5, T6, T7, T8>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9> SingleOutputStreamOperator<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>, ?> projectTuple10() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>> tType = new TupleTypeInfo<Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple10<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10> SingleOutputStreamOperator<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>, ?> projectTuple11() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>> tType = new TupleTypeInfo<Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple11<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11> SingleOutputStreamOperator<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>, ?> projectTuple12() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>> tType = new TupleTypeInfo<Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple12<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12> SingleOutputStreamOperator<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>, ?> projectTuple13() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>> tType = new TupleTypeInfo<Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple13<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13> SingleOutputStreamOperator<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>, ?> projectTuple14() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>> tType = new TupleTypeInfo<Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple14<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14> SingleOutputStreamOperator<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>, ?> projectTuple15() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>> tType = new TupleTypeInfo<Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple15<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15> SingleOutputStreamOperator<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>, ?> projectTuple16() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>> tType = new TupleTypeInfo<Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple16<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16> SingleOutputStreamOperator<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>, ?> projectTuple17() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>> tType = new TupleTypeInfo<Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple17<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17> SingleOutputStreamOperator<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>, ?> projectTuple18() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>> tType = new TupleTypeInfo<Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple18<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18> SingleOutputStreamOperator<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>, ?> projectTuple19() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>> tType = new TupleTypeInfo<Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple19<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19> SingleOutputStreamOperator<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>, ?> projectTuple20() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>> tType = new TupleTypeInfo<Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple20<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20> SingleOutputStreamOperator<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>, ?> projectTuple21() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>> tType = new TupleTypeInfo<Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple21<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21> SingleOutputStreamOperator<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>, ?> projectTuple22() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>> tType = new TupleTypeInfo<Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple22<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22> SingleOutputStreamOperator<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>, ?> projectTuple23() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>> tType = new TupleTypeInfo<Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple23<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23> SingleOutputStreamOperator<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>, ?> projectTuple24() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>> tType = new TupleTypeInfo<Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple24<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	/**
-	 * Projects a {@link Tuple} {@link DataStream} to the previously selected fields.
-	 *
-	 * @return The projected DataStream.
-	 * @see Tuple
-	 * @see DataStream
-	 */
-	public <T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24> SingleOutputStreamOperator<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>, ?> projectTuple25() {
-		TypeInformation<?>[] fTypes = extractFieldTypes(fieldIndexes, dataStream.getType());
-		TupleTypeInfo<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>> tType = new TupleTypeInfo<Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(fTypes);
-
-		return dataStream.transform("Projection", tType, new StreamProject<IN, Tuple25<T0, T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12, T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, T24>>(fieldIndexes, tType.createSerializer(dataStream.getExecutionConfig())));
-	}
-
-	public static TypeInformation<?>[] extractFieldTypes(int[] fields, TypeInformation<?> inType) {
-
-		TupleTypeInfo<?> inTupleType = (TupleTypeInfo<?>) inType;
-		TypeInformation<?>[] fieldTypes = new TypeInformation[fields.length];
-
-		for (int i = 0; i < fields.length; i++) {
-			fieldTypes[i] = inTupleType.getTypeAt(fields[i]);
-		}
-
-		return fieldTypes;
-	}
-
-}


[06/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
new file mode 100644
index 0000000..3355f1c
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.examples.join;
+
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Example illustrating join over sliding windows of streams in Flink.
+ *
+ * <p>
+ * This example will join two streams with a sliding window. One which emits grades and one which
+ * emits salaries of people. The input format for both sources has an additional timestamp
+ * as field 0. This is used to to event-time windowing. Time timestamps must be
+ * monotonically increasing.
+ *
+ * This example shows how to:
+ * <ul>
+ *   <li>do windowed joins,
+ *   <li>use tuple data types,
+ *   <li>write a simple streaming program.
+ * </ul>
+ */
+public class WindowJoin {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// obtain execution environment
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+		// connect to the data sources for grades and salaries
+		Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> input = getInputStreams(env);
+		DataStream<Tuple3<Long, String, Integer>> grades = input.f0;
+		DataStream<Tuple3<Long, String, Integer>> salaries = input.f1;
+
+		// extract the timestamps
+		grades = grades.assignTimestamps(new MyTimestampExtractor());
+		salaries = salaries.assignTimestamps(new MyTimestampExtractor());
+
+		// apply a temporal join over the two stream based on the names over one
+		// second windows
+		DataStream<Tuple3<String, Integer, Integer>> joinedStream = grades
+				.join(salaries)
+				.where(new NameKeySelector())
+				.equalTo(new NameKeySelector())
+				.window(TumblingTimeWindows.of(Time.of(5, TimeUnit.MILLISECONDS)))
+				.apply(new MyJoinFunction());
+
+		// emit result
+		if (fileOutput) {
+			joinedStream.writeAsText(outputPath, 1);
+		} else {
+			joinedStream.print();
+		}
+
+		// execute program
+		env.execute("Windowed Join Example");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	private final static String[] names = {"tom", "jerry", "alice", "bob", "john", "grace"};
+	private final static int GRADE_COUNT = 5;
+	private final static int SALARY_MAX = 10000;
+	private final static int SLEEP_TIME = 10;
+
+	/**
+	 * Continuously emit tuples with random names and integers (grades).
+	 */
+	public static class GradeSource implements SourceFunction<Tuple3<Long, String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		private Random rand;
+		private Tuple3<Long, String, Integer> outTuple;
+		private volatile boolean isRunning = true;
+		private int counter;
+
+		public GradeSource() {
+			rand = new Random();
+			outTuple = new Tuple3<>();
+		}
+
+		@Override
+		public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
+			while (isRunning && counter < 100) {
+				outTuple.f0 = System.currentTimeMillis();
+				outTuple.f1 = names[rand.nextInt(names.length)];
+				outTuple.f2 = rand.nextInt(GRADE_COUNT) + 1;
+				Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
+				counter++;
+				ctx.collect(outTuple);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+	}
+
+	/**
+	 * Continuously emit tuples with random names and integers (salaries).
+	 */
+	public static class SalarySource extends RichSourceFunction<Tuple3<Long, String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		private transient Random rand;
+		private transient Tuple3<Long, String, Integer> outTuple;
+		private volatile boolean isRunning;
+		private int counter;
+
+		public void open(Configuration parameters) throws Exception {
+			super.open(parameters);
+			rand = new Random();
+			outTuple = new Tuple3<Long, String, Integer>();
+			isRunning = true;
+		}
+
+
+		@Override
+		public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
+			while (isRunning && counter < 100) {
+				outTuple.f0 = System.currentTimeMillis();
+				outTuple.f1 = names[rand.nextInt(names.length)];
+				outTuple.f2 = rand.nextInt(SALARY_MAX) + 1;
+				Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
+				counter++;
+				ctx.collect(outTuple);
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+	}
+
+	public static class MySourceMap extends RichMapFunction<String, Tuple3<Long, String, Integer>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private String[] record;
+
+		public MySourceMap() {
+			record = new String[2];
+		}
+
+		@Override
+		public Tuple3<Long, String, Integer> map(String line) throws Exception {
+			record = line.substring(1, line.length() - 1).split(",");
+			return new Tuple3<>(Long.parseLong(record[0]), record[1], Integer.parseInt(record[2]));
+		}
+	}
+
+	public static class MyJoinFunction
+			implements
+			JoinFunction<Tuple3<Long, String, Integer>, Tuple3<Long, String, Integer>, Tuple3<String, Integer, Integer>> {
+
+		private static final long serialVersionUID = 1L;
+
+		private Tuple3<String, Integer, Integer> joined = new Tuple3<>();
+
+		@Override
+		public Tuple3<String, Integer, Integer> join(Tuple3<Long, String, Integer> first,
+				Tuple3<Long, String, Integer> second) throws Exception {
+			joined.f0 = first.f1;
+			joined.f1 = first.f2;
+			joined.f2 = second.f2;
+			return joined;
+		}
+	}
+
+	private static class MyTimestampExtractor implements TimestampExtractor<Tuple3<Long, String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long extractTimestamp(Tuple3<Long, String, Integer> element, long currentTimestamp) {
+			return element.f0;
+		}
+
+		@Override
+		public long extractWatermark(Tuple3<Long, String, Integer> element, long currentTimestamp) {
+			return element.f0 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+
+	private static class NameKeySelector implements KeySelector<Tuple3<Long, String, Integer>, String> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public String getKey(Tuple3<Long, String, Integer> value) throws Exception {
+			return value.f1;
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileInput = false;
+	private static boolean fileOutput = false;
+
+	private static String gradesPath;
+	private static String salariesPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			if (args.length == 1) {
+				fileOutput = true;
+				outputPath = args[0];
+			} else if (args.length == 3) {
+				fileInput = true;
+				fileOutput = true;
+				gradesPath = args[0];
+				salariesPath = args[1];
+				outputPath = args[2];
+			} else {
+				System.err.println("Usage: WindowJoin <result path> or WindowJoin <input path 1> <input path 2> " +
+						"<result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing WindowJoin with generated data.");
+			System.out.println("  Provide parameter to write to file.");
+			System.out.println("  Usage: WindowJoin <result path>");
+		}
+		return true;
+	}
+
+	private static Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> getInputStreams(
+			StreamExecutionEnvironment env) {
+
+		DataStream<Tuple3<Long, String, Integer>> grades;
+		DataStream<Tuple3<Long, String, Integer>> salaries;
+
+		if (fileInput) {
+			grades = env.readTextFile(gradesPath).map(new MySourceMap());
+			salaries = env.readTextFile(salariesPath).map(new MySourceMap());
+		} else {
+			grades = env.addSource(new GradeSource());
+			salaries = env.addSource(new SalarySource());
+		}
+
+		return Tuple2.of(grades, salaries);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
new file mode 100644
index 0000000..15c1280
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.examples.join.util;
+
+public class WindowJoinData {
+
+	public static final String GRADES_INPUT = "(0,john,5)\n" + "(0,tom,3)\n" + "(0,alice,1)\n" + "(0,grace,5)\n" +
+			"(1,john,4)\n" + "(1,bob,1)\n" + "(1,alice,2)\n" + "(1,alice,3)\n" + "(1,bob,5)\n" + "(1,alice,3)\n" + "(1,tom,5)\n" +
+			"(2,john,2)\n" + "(2,john,1)\n" + "(2,grace,2)\n" + "(2,jerry,2)\n" + "(2,tom,4)\n" + "(2,bob,4)\n" + "(2,bob,2)\n" +
+			"(3, tom,2)\n" + "(3,alice,5)\n" + "(3,grace,5)\n" + "(3,grace,1)\n" + "(3,alice,1)\n" + "(3,grace,3)\n" + "(3,tom,1)\n" +
+			"(4,jerry,5)\n" + "(4,john,3)\n" + "(4,john,4)\n" + "(4,john,1)\n" + "(4,jerry,3)\n" + "(4,grace,3)\n" + "(4,bob,3)\n" +
+			"(5,john,3)\n" + "(5,jerry,4)\n" + "(5,tom,5)\n" + "(5,tom,4)\n" + "(5,john,2)\n" + "(5,jerry,1)\n" + "(5,bob,1)\n" +
+			"(6,john,5)\n" + "(6,grace,4)\n" + "(6,tom,5)\n" + "(6,john,4)\n" + "(6,tom,1)\n" + "(6,grace,1)\n" + "(6,john,2)\n" +
+			"(7,jerry,3)\n" + "(7,jerry,5)\n" + "(7,tom,2)\n" + "(7,tom,2)\n" + "(7,alice,4)\n" + "(7,tom,4)\n" + "(7,jerry,4)\n" +
+			"(8,john,3)\n" + "(8,grace,4)\n" + "(8,tom,3)\n" + "(8,jerry,4)\n" + "(8,john,5)\n" + "(8,john,4)\n" + "(8,jerry,1)\n" +
+			"(9,john,5)\n" + "(9,alice,2)\n" + "(9,tom,1)\n" + "(9,alice,5)\n" + "(9,grace,4)\n" + "(9,bob,4)\n" + "(9,jerry,1)\n" +
+			"(10,john,5)\n" + "(10,tom,4)\n" + "(10,tom,5)\n" + "(10,jerry,5)\n" + "(10,tom,1)\n" + "(10,grace,3)\n" + "(10,bob,5)\n" +
+			"(11,john,1)\n" + "(11,alice,1)\n" + "(11,grace,3)\n" + "(11,grace,1)\n" + "(11,jerry,1)\n" + "(11,jerry,4)\n" +
+			"(12,bob,4)\n" + "(12,alice,3)\n" + "(12,tom,5)\n" + "(12,alice,4)\n" + "(12,alice,4)\n" + "(12,grace,4)\n" + "(12,john,5)\n" +
+			"(13,john,5)\n" + "(13,grace,4)\n" + "(13,tom,4)\n" + "(13,john,4)\n" + "(13,john,5)\n" + "(13,alice,5)\n" + "(13,jerry,5)\n" +
+			"(14,john,3)\n" + "(14,tom,5)\n" + "(14,jerry,4)\n" + "(14,grace,4)\n" + "(14,john,3)\n" + "(14,bob,2)";
+
+	public static final String SALARIES_INPUT = "(0,john,6469)\n" + "(0,jerry,6760)\n" + "(0,jerry,8069)\n" +
+			"(1,tom,3662)\n" + "(1,grace,8427)\n" + "(1,john,9425)\n" + "(1,bob,9018)\n" + "(1,john,352)\n" + "(1,tom,3770)\n" +
+			"(2,grace,7622)\n" + "(2,jerry,7441)\n" + "(2,alice,1468)\n" + "(2,bob,5472)\n" + "(2,grace,898)\n" +
+			"(3,tom,3849)\n" + "(3,grace,1865)\n" + "(3,alice,5582)\n" + "(3,john,9511)\n" + "(3,alice,1541)\n" +
+			"(4,john,2477)\n" + "(4,grace,3561)\n" + "(4,john,1670)\n" + "(4,grace,7290)\n" + "(4,grace,6565)\n" +
+			"(5,tom,6179)\n" + "(5,tom,1601)\n" + "(5,john,2940)\n" + "(5,bob,4685)\n" + "(5,bob,710)\n" + "(5,bob,5936)\n" +
+			"(6,jerry,1412)\n" + "(6,grace,6515)\n" + "(6,grace,3321)\n" + "(6,tom,8088)\n" + "(6,john,2876)\n" +
+			"(7,bob,9896)\n" + "(7,grace,7368)\n" + "(7,grace,9749)\n" + "(7,bob,2048)\n" + "(7,alice,4782)\n" +
+			"(8,alice,3375)\n" + "(8,tom,5841)\n" + "(8,bob,958)\n" + "(8,bob,5258)\n" + "(8,tom,3935)\n" + "(8,jerry,4394)\n" +
+			"(9,alice,102)\n" + "(9,alice,4931)\n" + "(9,alice,5240)\n" + "(9,jerry,7951)\n" + "(9,john,5675)\n" +
+			"(10,bob,609)\n" + "(10,alice,5997)\n" + "(10,jerry,9651)\n" + "(10,alice,1328)\n" + "(10,bob,1022)\n" +
+			"(11,grace,2578)\n" + "(11,jerry,9704)\n" + "(11,tom,4476)\n" + "(11,grace,3784)\n" + "(11,alice,6144)\n" +
+			"(12,bob,6213)\n" + "(12,alice,7525)\n" + "(12,jerry,2908)\n" + "(12,grace,8464)\n" + "(12,jerry,9920)\n" +
+			"(13,bob,3720)\n" + "(13,bob,7612)\n" + "(13,alice,7211)\n" + "(13,jerry,6484)\n" + "(13,alice,1711)\n" +
+			"(14,jerry,5994)\n" + "(14,grace,928)\n" + "(14,jerry,2492)\n" + "(14,grace,9080)\n" + "(14,tom,4330)\n" +
+			"(15,bob,8302)\n" + "(15,john,4981)\n" + "(15,tom,1781)\n" + "(15,grace,1379)\n" + "(15,jerry,3700)\n" +
+			"(16,jerry,3584)\n" + "(16,jerry,2038)\n" + "(16,jerry,3902)\n" + "(16,tom,1336)\n" + "(16,jerry,7500)\n" +
+			"(17,tom,3648)\n" + "(17,alice,2533)\n" + "(17,tom,8685)\n" + "(17,bob,3968)\n" + "(17,tom,3241)\n" + "(17,bob,7461)\n" +
+			"(18,jerry,2138)\n" + "(18,alice,7503)\n" + "(18,alice,6424)\n" + "(18,tom,140)\n" + "(18,john,9802)\n" +
+			"(19,grace,2977)\n" + "(19,grace,889)\n" + "(19,john,1338)";
+
+	private WindowJoinData() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
new file mode 100644
index 0000000..ce227e4
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/IncrementalLearningSkeleton.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.ml;
+
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.co.CoMapFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.util.Collector;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Skeleton for incremental machine learning algorithm consisting of a
+ * pre-computed model, which gets updated for the new inputs and new input data
+ * for which the job provides predictions.
+ *
+ * <p>
+ * This may serve as a base of a number of algorithms, e.g. updating an
+ * incremental Alternating Least Squares model while also providing the
+ * predictions.
+ *
+ * <p/>
+ * This example shows how to use:
+ * <ul>
+ *   <li>Connected streams
+ *   <li>CoFunctions
+ *   <li>Tuple data types
+ * </ul>
+ */
+public class IncrementalLearningSkeleton {
+
+	private static DataStream<Integer> trainingData = null;
+	private static DataStream<Integer> newData = null;
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+		trainingData = env.addSource(new FiniteTrainingDataSource());
+		newData = env.addSource(new FiniteNewDataSource());
+
+		// build new model on every second of new data
+		DataStream<Double[]> model = trainingData
+				.assignTimestamps(new LinearTimestamp())
+				.timeWindowAll(Time.of(5000, TimeUnit.MILLISECONDS))
+				.apply(new PartialModelBuilder());
+
+		// use partial model for newData
+		DataStream<Integer> prediction = newData.connect(model).map(new Predictor());
+
+		// emit result
+		if (fileOutput) {
+			prediction.writeAsText(outputPath, 1);
+		} else {
+			prediction.print();
+		}
+
+		// execute program
+		env.execute("Streaming Incremental Learning");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Feeds new data for newData. By default it is implemented as constantly
+	 * emitting the Integer 1 in a loop.
+	 */
+	public static class FiniteNewDataSource implements SourceFunction<Integer> {
+		private static final long serialVersionUID = 1L;
+		private int counter;
+
+		@Override
+		public void run(SourceContext<Integer> ctx) throws Exception {
+			Thread.sleep(15);
+			while (counter < 50) {
+				ctx.collect(getNewData());
+			}
+		}
+
+		@Override
+		public void cancel() {
+			// No cleanup needed
+		}
+
+		private Integer getNewData() throws InterruptedException {
+			Thread.sleep(5);
+			counter++;
+			return 1;
+		}
+	}
+
+	/**
+	 * Feeds new training data for the partial model builder. By default it is
+	 * implemented as constantly emitting the Integer 1 in a loop.
+	 */
+	public static class FiniteTrainingDataSource implements SourceFunction<Integer> {
+		private static final long serialVersionUID = 1L;
+		private int counter = 0;
+
+		@Override
+		public void run(SourceContext<Integer> collector) throws Exception {
+			while (counter < 8200) {
+				collector.collect(getTrainingData());
+			}
+		}
+
+		@Override
+		public void cancel() {
+			// No cleanup needed
+		}
+
+		private Integer getTrainingData() throws InterruptedException {
+			counter++;
+			return 1;
+		}
+	}
+
+	public static class LinearTimestamp implements TimestampExtractor<Integer> {
+		private static final long serialVersionUID = 1L;
+
+		private long counter = 0L;
+
+		@Override
+		public long extractTimestamp(Integer element, long currentTimestamp) {
+			return counter += 10L;
+		}
+
+		@Override
+		public long extractWatermark(Integer element, long currentTimestamp) {
+			return counter - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+
+	}
+
+	/**
+	 * Builds up-to-date partial models on new training data.
+	 */
+	public static class PartialModelBuilder implements AllWindowFunction<Integer, Double[], TimeWindow> {
+		private static final long serialVersionUID = 1L;
+
+		protected Double[] buildPartialModel(Iterable<Integer> values) {
+			return new Double[]{1.};
+		}
+
+		@Override
+		public void apply(TimeWindow window, Iterable<Integer> values, Collector<Double[]> out) throws Exception {
+			out.collect(buildPartialModel(values));
+		}
+	}
+
+	/**
+	 * Creates newData using the model produced in batch-processing and the
+	 * up-to-date partial model.
+	 * <p/>
+	 * <p>
+	 * By defaults emits the Integer 0 for every newData and the Integer 1
+	 * for every model update.
+	 * </p>
+	 */
+	public static class Predictor implements CoMapFunction<Integer, Double[], Integer> {
+		private static final long serialVersionUID = 1L;
+
+		Double[] batchModel = null;
+		Double[] partialModel = null;
+
+		@Override
+		public Integer map1(Integer value) {
+			// Return newData
+			return predict(value);
+		}
+
+		@Override
+		public Integer map2(Double[] value) {
+			// Update model
+			partialModel = value;
+			batchModel = getBatchModel();
+			return 1;
+		}
+
+		// pulls model built with batch-job on the old training data
+		protected Double[] getBatchModel() {
+			return new Double[]{0.};
+		}
+
+		// performs newData using the two models
+		protected Integer predict(Integer inTuple) {
+			return 0;
+		}
+
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 1) {
+				outputPath = args[0];
+			} else {
+				System.err.println("Usage: IncrementalLearningSkeleton <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing IncrementalLearningSkeleton with generated data.");
+			System.out.println("  Provide parameter to write to file.");
+			System.out.println("  Usage: IncrementalLearningSkeleton <result path>");
+		}
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
new file mode 100644
index 0000000..8a6cd88
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/ml/util/IncrementalLearningSkeletonData.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.ml.util;
+
+public class IncrementalLearningSkeletonData {
+
+	public static final String RESULTS = "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" +
+			"1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "1\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" + "0\n" +
+			"0\n" + "0\n" + "0\n" + "0\n";
+
+	private IncrementalLearningSkeletonData() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
new file mode 100644
index 0000000..17add2c
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/socket/SocketTextStreamWordCount.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.examples.socket;
+
+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.examples.wordcount.WordCount.Tokenizer;
+
+/**
+ * This example shows an implementation of WordCount with data from a text
+ * socket. To run the example make sure that the service providing the text data
+ * is already up and running.
+ * <p/>
+ * <p/>
+ * To start an example socket text stream on your local machine run netcat from
+ * a command line: <code>nc -lk 9999</code>, where the parameter specifies the
+ * port number.
+ * <p/>
+ * <p/>
+ * <p/>
+ * Usage:
+ * <code>SocketTextStreamWordCount &lt;hostname&gt; &lt;port&gt; &lt;result path&gt;</code>
+ * <br>
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>use StreamExecutionEnvironment.socketTextStream
+ * <li>write a simple Flink program,
+ * <li>write and use user-defined functions.
+ * </ul>
+ *
+ * @see <a href="www.openbsd.org/cgi-bin/man.cgi?query=nc">netcat</a>
+ */
+public class SocketTextStreamWordCount {
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment
+				.getExecutionEnvironment();
+
+		// get input data
+		DataStream<String> text = env.socketTextStream(hostName, port, '\n', 0);
+
+		DataStream<Tuple2<String, Integer>> counts =
+				// split up the lines in pairs (2-tuples) containing: (word,1)
+				text.flatMap(new Tokenizer())
+						// group by the tuple field "0" and sum up tuple field "1"
+						.keyBy(0)
+						.sum(1);
+
+		if (fileOutput) {
+			counts.writeAsText(outputPath, 1);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("WordCount from SocketTextStream Example");
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String hostName;
+	private static int port;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		// parse input arguments
+		if (args.length == 3) {
+			fileOutput = true;
+			hostName = args[0];
+			port = Integer.valueOf(args[1]);
+			outputPath = args[2];
+		} else if (args.length == 2) {
+			hostName = args[0];
+			port = Integer.valueOf(args[1]);
+		} else {
+			System.err.println("Usage: SocketTextStreamWordCount <hostname> <port> [<output path>]");
+			return false;
+		}
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
new file mode 100644
index 0000000..c2477b5
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/TwitterStream.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.twitter;
+
+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.connectors.json.JSONParseFlatMap;
+import org.apache.flink.streaming.connectors.twitter.TwitterSource;
+import org.apache.flink.streaming.examples.twitter.util.TwitterStreamData;
+import org.apache.flink.util.Collector;
+import org.apache.sling.commons.json.JSONException;
+
+import java.util.StringTokenizer;
+
+/**
+ * Implements the "TwitterStream" program that computes a most used word
+ * occurrence over JSON files in a streaming fashion.
+ * <p/>
+ * <p/>
+ * The input is a JSON text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>TwitterStream &lt;text path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from
+ * {@link TwitterStreamData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>acquire external data,
+ * <li>use in-line defined functions,
+ * <li>handle flattened stream inputs.
+ * </ul>
+ */
+public class TwitterStream {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		DataStream<String> streamSource = getTextDataStream(env);
+
+		DataStream<Tuple2<String, Integer>> tweets = streamSource
+				// selecting English tweets and splitting to (word, 1)
+				.flatMap(new SelectEnglishAndTokenizeFlatMap())
+				// group by words and sum their occurrences
+				.keyBy(0).sum(1);
+
+		// emit result
+		if (fileOutput) {
+			tweets.writeAsText(outputPath);
+		} else {
+			tweets.print();
+		}
+
+		// execute program
+		env.execute("Twitter Streaming Example");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Makes sentences from English tweets.
+	 * <p/>
+	 * <p>
+	 * Implements a string tokenizer that splits sentences into words as a
+	 * user-defined FlatMapFunction. The function takes a line (String) and
+	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
+	 * Integer>).
+	 * </p>
+	 */
+	public static class SelectEnglishAndTokenizeFlatMap extends JSONParseFlatMap<String, Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		/**
+		 * Select the language from the incoming JSON text
+		 */
+		@Override
+		public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception {
+			try {
+				if (getString(value, "user.lang").equals("en")) {
+					// message of tweet
+					StringTokenizer tokenizer = new StringTokenizer(getString(value, "text"));
+
+					// split the message
+					while (tokenizer.hasMoreTokens()) {
+						String result = tokenizer.nextToken().replaceAll("\\s*", "").toLowerCase();
+
+						if (result != null && !result.equals("")) {
+							out.collect(new Tuple2<String, Integer>(result, 1));
+						}
+					}
+				}
+			} catch (JSONException e) {
+				// the JSON was not parsed correctly
+			}
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileInput = false;
+	private static boolean fileOutput = false;
+	private static String propertiesPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				fileInput = true;
+				propertiesPath = args[0];
+				outputPath = args[1];
+			} else if (args.length == 1) {
+				outputPath = args[0];
+			} else {
+				System.err.println("USAGE:\nTwitterStream [<pathToPropertiesFile>] <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing TwitterStream example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from a file.");
+			System.out.println("  USAGE: TwitterStream [<pathToPropertiesFile>] <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
+		if (fileInput) {
+			// read the text file from given input path
+			return env.addSource(new TwitterSource(propertiesPath));
+		} else {
+			// get default test text data
+			return env.fromElements(TwitterStreamData.TEXTS);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java
new file mode 100644
index 0000000..b06d193
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/twitter/util/TwitterStreamData.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.twitter.util;
+
+//example data looking like tweets, but not acquired from Twitter
+public class TwitterStreamData {
+	public static final String[] TEXTS = new String[] {
+			"{\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"id\":000000000000000000,\"id_str\":\"000000000000000000\",\"text\":\"Apache Flink\",\"source\":null,\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":0000000000,\"id_str\":\"0000000000\",\"name\":\"Apache Flink\",\"screen_name\":\"Apache Flink\",\"location\":\"Berlin\",\"protected\":false,\"verified\":false,\"followers_count\":999999,\"friends_count\":99999,\"listed_count\":999,\"favourites_count\":9999,\"statuses_count\":999,\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"utc_offset\":7200,\"time_zone\":\"Amsterdam\",\"geo_enabled\":false,\"lang\":\"en\",\"entities\":{\"hashtags\":[{\"text\":\"example1\",\"indices\":[0,0]},{\"text\":\"tweet1\",\"indices\":[0,0]}]},\"contributors_enabled\":false,\"is_translator\":false,\"profile_background_color\":\"C6E2EE\",\"profile_background_
 tile\":false,\"profile_link_color\":\"1F98C7\",\"profile_sidebar_border_color\":\"FFFFFF\",\"profile_sidebar_fill_color\":\"252429\",\"profile_text_color\":\"666666\",\"profile_use_background_image\":true,\"default_profile\":false,\"default_profile_image\":false,\"following\":null,\"follow_request_sent\":null,\"notifications\":null},\"geo\":null,\"coordinates\":null,\"place\":null,\"contributors\":null}",
+			"{\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"id\":000000000000000001,\"id_str\":\"000000000000000000\",\"text\":\"Apache Flink\",\"source\":null,\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":0000000000,\"id_str\":\"0000000000\",\"name\":\"Apache Flink\",\"screen_name\":\"Apache Flink\",\"location\":\"Berlin\",\"protected\":false,\"verified\":false,\"followers_count\":999999,\"friends_count\":99999,\"listed_count\":999,\"favourites_count\":9999,\"statuses_count\":999,\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"utc_offset\":7200,\"time_zone\":\"Amsterdam\",\"geo_enabled\":false,\"lang\":\"en\",\"entities\":{\"hashtags\":[{\"text\":\"example2\",\"indices\":[0,0]},{\"text\":\"tweet2\",\"indices\":[0,0]}]},\"contributors_enabled\":false,\"is_translator\":false,\"profile_background_color\":\"C6E2EE\",\"profile_background_
 tile\":false,\"profile_link_color\":\"1F98C7\",\"profile_sidebar_border_color\":\"FFFFFF\",\"profile_sidebar_fill_color\":\"252429\",\"profile_text_color\":\"666666\",\"profile_use_background_image\":true,\"default_profile\":false,\"default_profile_image\":false,\"following\":null,\"follow_request_sent\":null,\"notifications\":null},\"geo\":null,\"coordinates\":null,\"place\":null,\"contributors\":null}",
+			"{\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"id\":000000000000000002,\"id_str\":\"000000000000000000\",\"text\":\"Apache Flink\",\"source\":null,\"truncated\":false,\"in_reply_to_status_id\":null,\"in_reply_to_status_id_str\":null,\"in_reply_to_user_id\":null,\"in_reply_to_user_id_str\":null,\"in_reply_to_screen_name\":null,\"user\":{\"id\":0000000000,\"id_str\":\"0000000000\",\"name\":\"Apache Flink\",\"screen_name\":\"Apache Flink\",\"location\":\"Berlin\",\"protected\":false,\"verified\":false,\"followers_count\":999999,\"friends_count\":99999,\"listed_count\":999,\"favourites_count\":9999,\"statuses_count\":999,\"created_at\":\"Mon Jan 1 00:00:00 +0000 1901\",\"utc_offset\":7200,\"time_zone\":\"Amsterdam\",\"geo_enabled\":false,\"lang\":\"en\",\"entities\":{\"hashtags\":[{\"text\":\"example3\",\"indices\":[0,0]},{\"text\":\"tweet3\",\"indices\":[0,0]}]},\"contributors_enabled\":false,\"is_translator\":false,\"profile_background_color\":\"C6E2EE\",\"profile_background_
 tile\":false,\"profile_link_color\":\"1F98C7\",\"profile_sidebar_border_color\":\"FFFFFF\",\"profile_sidebar_fill_color\":\"252429\",\"profile_text_color\":\"666666\",\"profile_use_background_image\":true,\"default_profile\":false,\"default_profile_image\":false,\"following\":null,\"follow_request_sent\":null,\"notifications\":null},\"geo\":null,\"coordinates\":null,\"place\":null,\"contributors\":null}",
+	};
+
+	public static final String STREAMING_COUNTS_AS_TUPLES = "(apache,1)\n" + "(apache,2)\n" + "(apache,3)\n" + "(flink,1)\n" + "(flink,2)\n" + "(flink,3)\n";
+
+	private TwitterStreamData() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
new file mode 100644
index 0000000..982b73d
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.windowing;
+
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+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.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+@SuppressWarnings("serial")
+public class GroupedProcessingTimeWindowExample {
+	
+	public static void main(String[] args) throws Exception {
+		
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setParallelism(4);
+		
+		DataStream<Tuple2<Long, Long>> stream = env
+				.addSource(new RichParallelSourceFunction<Tuple2<Long, Long>>() {
+					
+					private volatile boolean running = true;
+					
+					@Override
+					public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
+						
+						final long startTime = System.currentTimeMillis();
+						
+						final long numElements = 20000000;
+						final long numKeys = 10000;
+						long val = 1L;
+						long count = 0L;
+						
+						
+						while (running && count < numElements) {
+							count++;
+							ctx.collect(new Tuple2<Long, Long>(val++, 1L));
+							
+							if (val > numKeys) {
+								val = 1L;
+							}
+						}
+
+						final long endTime = System.currentTimeMillis();
+						System.out.println("Took " + (endTime-startTime) + " msecs for " + numElements + " values");
+					}
+
+					@Override
+					public void cancel() {
+						running = false;
+					}
+				});
+		
+		stream
+			.keyBy(0)
+			.timeWindow(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
+			.reduce(new SummingReducer())
+
+			// alternative: use a apply function which does not pre-aggregate
+//			.keyBy(new FirstFieldKeyExtractor<Tuple2<Long, Long>, Long>())
+//			.window(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
+//			.apply(new SummingWindowFunction())
+				
+			.addSink(new SinkFunction<Tuple2<Long, Long>>() {
+				@Override
+				public void invoke(Tuple2<Long, Long> value) {
+				}
+			});
+		
+		env.execute();
+	}
+	
+	public static class FirstFieldKeyExtractor<Type extends Tuple, Key> implements KeySelector<Type, Key> {
+		
+		@Override
+		@SuppressWarnings("unchecked")
+		public Key getKey(Type value) {
+			return (Key) value.getField(0);
+		}
+	}
+
+	public static class SummingWindowFunction implements WindowFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Long, Window> {
+
+		@Override
+		public void apply(Long key, Window window, Iterable<Tuple2<Long, Long>> values, Collector<Tuple2<Long, Long>> out) {
+			long sum = 0L;
+			for (Tuple2<Long, Long> value : values) {
+				sum += value.f1;
+			}
+
+			out.collect(new Tuple2<>(key, sum));
+		}
+	}
+
+	public static class SummingReducer implements ReduceFunction<Tuple2<Long, Long>> {
+
+		@Override
+		public Tuple2<Long, Long> reduce(Tuple2<Long, Long> value1, Tuple2<Long, Long> value2) {
+			return new Tuple2<>(value1.f0, value1.f1 + value2.f1);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
new file mode 100644
index 0000000..3c63156
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.windowing;
+
+import org.apache.flink.api.common.state.OperatorState;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class SessionWindowing {
+
+	@SuppressWarnings("serial")
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(2);
+
+		final List<Tuple3<String, Long, Integer>> input = new ArrayList<>();
+
+		input.add(new Tuple3<>("a", 1L, 1));
+		input.add(new Tuple3<>("b", 1L, 1));
+		input.add(new Tuple3<>("b", 3L, 1));
+		input.add(new Tuple3<>("b", 5L, 1));
+		input.add(new Tuple3<>("c", 6L, 1));
+		// We expect to detect the session "a" earlier than this point (the old
+		// functionality can only detect here when the next starts)
+		input.add(new Tuple3<>("a", 10L, 1));
+		// We expect to detect session "b" and "c" at this point as well
+		input.add(new Tuple3<>("c", 11L, 1));
+
+		DataStream<Tuple3<String, Long, Integer>> source = env
+				.addSource(new EventTimeSourceFunction<Tuple3<String,Long,Integer>>() {
+					private static final long serialVersionUID = 1L;
+
+					@Override
+					public void run(SourceContext<Tuple3<String, Long, Integer>> ctx) throws Exception {
+						for (Tuple3<String, Long, Integer> value : input) {
+							ctx.collectWithTimestamp(value, value.f1);
+							ctx.emitWatermark(new Watermark(value.f1 - 1));
+							if (!fileOutput) {
+								System.out.println("Collected: " + value);
+							}
+						}
+						ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
+					}
+
+					@Override
+					public void cancel() {
+					}
+				});
+
+		// We create sessions for each id with max timeout of 3 time units
+		DataStream<Tuple3<String, Long, Integer>> aggregated = source
+				.keyBy(0)
+				.window(GlobalWindows.create())
+				.trigger(new SessionTrigger(3L))
+				.sum(2);
+
+		if (fileOutput) {
+			aggregated.writeAsText(outputPath);
+		} else {
+			aggregated.print();
+		}
+
+		env.execute();
+	}
+
+	private static class SessionTrigger implements Trigger<Tuple3<String, Long, Integer>, GlobalWindow> {
+
+		private static final long serialVersionUID = 1L;
+
+		private final Long sessionTimeout;
+
+		public SessionTrigger(Long sessionTimeout) {
+			this.sessionTimeout = sessionTimeout;
+
+		}
+
+		@Override
+		public TriggerResult onElement(Tuple3<String, Long, Integer> element, long timestamp, GlobalWindow window, TriggerContext ctx) throws Exception {
+
+			OperatorState<Long> lastSeenState = ctx.getKeyValueState("last-seen", 1L);
+			Long lastSeen = lastSeenState.value();
+
+			Long timeSinceLastEvent = timestamp - lastSeen;
+
+			// Update the last seen event time
+			lastSeenState.update(timestamp);
+
+			ctx.registerEventTimeTimer(lastSeen + sessionTimeout);
+
+			if (timeSinceLastEvent > sessionTimeout) {
+				return TriggerResult.FIRE_AND_PURGE;
+			} else {
+				return TriggerResult.CONTINUE;
+			}
+		}
+
+		@Override
+		public TriggerResult onEventTime(long time, TriggerContext ctx) throws Exception {
+			OperatorState<Long> lastSeenState = ctx.getKeyValueState("last-seen", 1L);
+			Long lastSeen = lastSeenState.value();
+
+			if (time - lastSeen >= sessionTimeout) {
+				return TriggerResult.FIRE_AND_PURGE;
+			}
+			return TriggerResult.CONTINUE;
+		}
+
+		@Override
+		public TriggerResult onProcessingTime(long time,
+				TriggerContext ctx) throws Exception {
+			return TriggerResult.CONTINUE;
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			if (args.length == 1) {
+				fileOutput = true;
+				outputPath = args[0];
+			} else {
+				System.err.println("Usage: SessionWindowing <result path>");
+				return false;
+			}
+		}
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
new file mode 100644
index 0000000..df3402e
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.examples.windowing;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
+import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger;
+
+import java.util.Arrays;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * An example of grouped stream windowing where different eviction and trigger
+ * policies can be used. A source fetches events from cars every 1 sec
+ * containing their id, their current speed (kmh), overall elapsed distance (m)
+ * and a timestamp. The streaming example triggers the top speed of each car
+ * every x meters elapsed for the last y seconds.
+ */
+public class TopSpeedWindowing {
+
+	private static final int NUM_CAR_EVENTS = 100;
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+		@SuppressWarnings({"rawtypes", "serial"})
+		DataStream<Tuple4<Integer, Integer, Double, Long>> carData;
+
+		if (fileInput) {
+			carData = env.readTextFile(inputPath).map(new ParseCarData());
+		} else {
+			carData = env.addSource(CarSource.create(numOfCars));
+		}
+
+		DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds = carData
+				.assignTimestamps(new CarTimestamp())
+				.keyBy(0)
+				.window(GlobalWindows.create())
+				.evictor(TimeEvictor.of(Time.of(evictionSec, TimeUnit.SECONDS)))
+				.trigger(DeltaTrigger.of(triggerMeters,
+						new DeltaFunction<Tuple4<Integer, Integer, Double, Long>>() {
+							private static final long serialVersionUID = 1L;
+
+							@Override
+							public double getDelta(
+									Tuple4<Integer, Integer, Double, Long> oldDataPoint,
+									Tuple4<Integer, Integer, Double, Long> newDataPoint) {
+								return newDataPoint.f2 - oldDataPoint.f2;
+							}
+						}))
+				.maxBy(1);
+
+		if (fileOutput) {
+			topSpeeds.print();
+			topSpeeds.writeAsText(outputPath);
+		} else {
+			topSpeeds.print();
+		}
+
+		env.execute("CarTopSpeedWindowingExample");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	private static class CarSource implements SourceFunction<Tuple4<Integer, Integer, Double, Long>> {
+
+		private static final long serialVersionUID = 1L;
+		private Integer[] speeds;
+		private Double[] distances;
+
+		private Random rand = new Random();
+
+		private volatile boolean isRunning = true;
+		private int counter;
+
+		private CarSource(int numOfCars) {
+			speeds = new Integer[numOfCars];
+			distances = new Double[numOfCars];
+			Arrays.fill(speeds, 50);
+			Arrays.fill(distances, 0d);
+		}
+
+		public static CarSource create(int cars) {
+			return new CarSource(cars);
+		}
+
+		@Override
+		public void run(SourceContext<Tuple4<Integer, Integer, Double, Long>> ctx) throws Exception {
+
+			while (isRunning && counter < NUM_CAR_EVENTS) {
+				Thread.sleep(100);
+				for (int carId = 0; carId < speeds.length; carId++) {
+					if (rand.nextBoolean()) {
+						speeds[carId] = Math.min(100, speeds[carId] + 5);
+					} else {
+						speeds[carId] = Math.max(0, speeds[carId] - 5);
+					}
+					distances[carId] += speeds[carId] / 3.6d;
+					Tuple4<Integer, Integer, Double, Long> record = new Tuple4<Integer, Integer, Double, Long>(carId,
+							speeds[carId], distances[carId], System.currentTimeMillis());
+					ctx.collect(record);
+					counter++;
+				}
+			}
+		}
+
+		@Override
+		public void cancel() {
+			isRunning = false;
+		}
+	}
+
+	private static class ParseCarData extends
+			RichMapFunction<String, Tuple4<Integer, Integer, Double, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple4<Integer, Integer, Double, Long> map(String record) {
+			String rawData = record.substring(1, record.length() - 1);
+			String[] data = rawData.split(",");
+			return new Tuple4<>(Integer.valueOf(data[0]), Integer.valueOf(data[1]), Double.valueOf(data[2]), Long.valueOf(data[3]));
+		}
+	}
+
+	private static class CarTimestamp implements TimestampExtractor<Tuple4<Integer, Integer, Double, Long>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public long extractTimestamp(Tuple4<Integer, Integer, Double, Long> element,
+				long currentTimestamp) {
+			return element.f3;
+		}
+
+		@Override
+		public long extractWatermark(Tuple4<Integer, Integer, Double, Long> element,
+				long currentTimestamp) {
+			return element.f3 - 1;
+		}
+
+		@Override
+		public long getCurrentWatermark() {
+			return Long.MIN_VALUE;
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileInput = false;
+	private static boolean fileOutput = false;
+	private static int numOfCars = 2;
+	private static int evictionSec = 10;
+	private static double triggerMeters = 50;
+	private static String inputPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			if (args.length == 2) {
+				fileInput = true;
+				fileOutput = true;
+				inputPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: TopSpeedWindowingExample <input path> <output path>");
+				return false;
+			}
+		}
+		return true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
new file mode 100644
index 0000000..f3d57bf
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.examples.windowing;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.examples.wordcount.WordCount;
+
+/**
+ * Implements a windowed version of the streaming "WordCount" program.
+ *
+ * <p>
+ * The input is a plain text file with lines separated by newline characters.
+ * 
+ * <p>
+ * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from
+ * {@link org.apache.flink.examples.java.wordcount.util.WordCountData}.
+ *
+ * <p>
+ * This example shows how to:
+ * <ul>
+ * <li>write a simple Flink Streaming program,
+ * <li>use tuple data types,
+ * <li>use basic windowing abstractions.
+ * </ul>
+ *
+ */
+public class WindowWordCount {
+
+	// window parameters with default values
+	private static int windowSize = 250;
+	private static int slideSize = 150;
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		DataStream<String> text = getTextDataStream(env);
+
+		DataStream<Tuple2<String, Integer>> counts =
+		// split up the lines in pairs (2-tuples) containing: (word,1)
+		text.flatMap(new WordCount.Tokenizer())
+				// create windows of windowSize records slided every slideSize records
+				.keyBy(0)
+				.countWindow(windowSize, slideSize)
+				// group by the tuple field "0" and sum up tuple field "1"
+				.sum(1);
+
+		// emit result
+		if (fileOutput) {
+			counts.writeAsText(outputPath);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("WindowWordCount");
+	}
+
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length >= 2 && args.length <= 4) {
+				textPath = args[0];
+				outputPath = args[1];
+				if (args.length >= 3){
+					windowSize = Integer.parseInt(args[2]);
+
+					// if no slide size is specified use the
+					slideSize = args.length == 3 ? windowSize : Integer.parseInt(args[2]);
+				}
+			} else {
+				System.err.println("Usage: WindowWordCount <text path> <result path> [<window size>] [<slide size>]");
+				return false;
+			}
+		} else {
+			System.out.println("Executing WindowWordCount example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from a file.");
+			System.out.println("  Usage: WindowWordCount <text path> <result path> [<window size>] [<slide size>]");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		} else {
+			// get default test text data
+			return env.fromElements(WordCountData.WORDS);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
new file mode 100644
index 0000000..c1a99a8
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.examples.windowing.util;
+
+public class SessionWindowingData {
+
+	public static final String EXPECTED = "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,1,3)\n" +
+			"(a,10,1)";
+
+	private SessionWindowingData() {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
new file mode 100644
index 0000000..bf63695
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.windowing.util;
+
+public class TopSpeedWindowingExampleData {
+
+	public static final String CAR_DATA =
+					"(0,55,15.277777777777777,1424951918630)\n" + "(1,45,12.5,1424951918632)\n" +
+					"(0,50,29.166666666666664,1424951919632)\n" + "(1,50,26.38888888888889,1424951919632)\n" +
+					"(0,55,44.44444444444444,1424951920633)\n" + "(1,45,38.888888888888886,1424951920633)\n" +
+					"(0,50,58.33333333333333,1424951921634)\n" + "(1,40,50.0,1424951921634)\n" +
+					"(0,55,73.6111111111111,1424951922634)\n" + "(1,35,59.72222222222222,1424951922634)\n" +
+					"(0,60,90.27777777777777,1424951923634)\n" + "(1,40,70.83333333333333,1424951923634)\n" +
+					"(0,65,108.33333333333333,1424951924635)\n" + "(1,35,80.55555555555554,1424951924635)\n" +
+					"(0,60,125.0,1424951925635)\n" + "(1,40,91.66666666666666,1424951925635)\n" +
+					"(0,55,140.27777777777777,1424951926635)\n" + "(1,45,104.16666666666666,1424951926636)\n" +
+					"(0,60,156.94444444444443,1424951927636)\n" + "(1,50,118.05555555555554,1424951927636)\n" +
+					"(0,55,172.2222222222222,1424951928636)\n" + "(1,45,130.55555555555554,1424951928636)\n" +
+					"(0,50,186.1111111111111,1424951929636)\n" + "(1,50,144.44444444444443,1424951929637)\n" +
+					"(0,55,201.38888888888886,1424951930637)\n" + "(1,55,159.7222222222222,1424951930637)\n" +
+					"(0,60,218.05555555555551,1424951931637)\n" + "(1,60,176.38888888888886,1424951931637)\n" +
+					"(0,55,233.3333333333333,1424951932637)\n" + "(1,65,194.4444444444444,1424951932638)\n" +
+					"(0,50,247.22222222222217,1424951933638)\n" + "(1,70,213.88888888888886,1424951933638)\n" +
+					"(0,45,259.7222222222222,1424951934638)\n" + "(1,65,231.9444444444444,1424951934638)\n" +
+					"(0,50,273.6111111111111,1424951935638)\n" + "(1,70,251.38888888888886,1424951935639)\n" +
+					"(0,55,288.88888888888886,1424951936639)\n" + "(1,75,272.2222222222222,1424951936639)\n" +
+					"(0,50,302.77777777777777,1424951937639)\n" + "(1,70,291.66666666666663,1424951937639)\n" +
+					"(0,45,315.27777777777777,1424951938640)\n" + "(1,65,309.7222222222222,1424951938640)\n" +
+					"(0,50,329.1666666666667,1424951939640)\n" + "(1,70,329.16666666666663,1424951939640)\n" +
+					"(0,55,344.44444444444446,1424951940640)\n" + "(1,65,347.2222222222222,1424951940640)\n" +
+					"(0,50,358.33333333333337,1424951941641)\n" + "(1,70,366.66666666666663,1424951941641)\n" +
+					"(0,55,373.61111111111114,1424951942641)\n" + "(1,65,384.7222222222222,1424951942641)\n" +
+					"(0,50,387.50000000000006,1424951943641)\n" + "(1,70,404.16666666666663,1424951943641)\n" +
+					"(0,45,400.00000000000006,1424951944642)\n" + "(1,65,422.2222222222222,1424951944642)\n" +
+					"(0,50,413.88888888888897,1424951945642)\n" + "(1,60,438.88888888888886,1424951945642)\n" +
+					"(0,45,426.38888888888897,1424951946642)\n" + "(1,65,456.9444444444444,1424951946642)\n" +
+					"(0,40,437.50000000000006,1424951947643)\n" + "(1,70,476.38888888888886,1424951947643)\n" +
+					"(0,45,450.00000000000006,1424951948643)\n" + "(1,75,497.2222222222222,1424951948643)\n" +
+					"(0,40,461.11111111111114,1424951949643)\n" + "(1,80,519.4444444444443,1424951949644)\n" +
+					"(0,45,473.61111111111114,1424951950644)\n" + "(1,75,540.2777777777777,1424951950644)\n" +
+					"(0,50,487.50000000000006,1424951951644)\n" + "(1,80,562.4999999999999,1424951951644)\n" +
+					"(0,45,500.00000000000006,1424951952644)\n" + "(1,85,586.111111111111,1424951952645)\n" +
+					"(0,40,511.11111111111114,1424951953645)\n" + "(1,80,608.3333333333331,1424951953645)\n" +
+					"(0,35,520.8333333333334,1424951954645)\n" + "(1,75,629.1666666666665,1424951954645)\n" +
+					"(0,40,531.9444444444445,1424951955645)\n" + "(1,70,648.611111111111,1424951955646)\n" +
+					"(0,45,544.4444444444445,1424951956646)\n" + "(1,75,669.4444444444443,1424951956646)\n" +
+					"(0,50,558.3333333333334,1424951957646)\n" + "(1,80,691.6666666666665,1424951957646)\n" +
+					"(0,55,573.6111111111112,1424951958646)\n" + "(1,85,715.2777777777776,1424951958647)\n" +
+					"(0,60,590.2777777777778,1424951959647)\n" + "(1,80,737.4999999999998,1424951959647)\n" +
+					"(0,65,608.3333333333334,1424951960647)\n" + "(1,85,761.1111111111109,1424951960647)\n" +
+					"(0,70,627.7777777777778,1424951961647)\n" + "(1,80,783.333333333333,1424951961648)\n" +
+					"(0,75,648.6111111111112,1424951962648)\n" + "(1,85,806.9444444444441,1424951962648)\n" +
+					"(0,80,670.8333333333334,1424951963648)\n" + "(1,90,831.9444444444441,1424951963648)\n" +
+					"(0,75,691.6666666666667,1424951964649)\n" + "(1,95,858.333333333333,1424951964649)\n" +
+					"(0,70,711.1111111111112,1424951965649)\n" + "(1,90,883.333333333333,1424951965649)\n" +
+					"(0,75,731.9444444444446,1424951966649)\n" + "(1,95,909.722222222222,1424951966649)\n" +
+					"(0,70,751.388888888889,1424951967649)\n" + "(1,100,937.4999999999998,1424951967650)\n" +
+					"(0,75,772.2222222222224,1424951968650)\n" + "(1,100,965.2777777777776,1424951968650)\n" +
+					"(0,80,794.4444444444446,1424951969650)\n" + "(1,100,993.0555555555554,1424951969650)\n" +
+					"(0,75,815.2777777777779,1424951970651)\n" + "(1,100,1020.8333333333333,1424951970651)\n" +
+					"(0,80,837.5000000000001,1424951971651)\n" + "(1,100,1048.611111111111,1424951971651)\n" +
+					"(0,85,861.1111111111112,1424951972651)\n" + "(1,100,1076.388888888889,1424951972651)\n" +
+					"(0,80,883.3333333333334,1424951973652)\n" + "(1,95,1102.7777777777778,1424951973652)\n" +
+					"(0,75,904.1666666666667,1424951974652)\n" + "(1,100,1130.5555555555557,1424951974652)\n" +
+					"(0,70,923.6111111111112,1424951975652)\n" + "(1,100,1158.3333333333335,1424951975652)\n" +
+					"(0,75,944.4444444444446,1424951976653)\n" + "(1,100,1186.1111111111113,1424951976653)\n" +
+					"(0,80,966.6666666666667,1424951977653)\n" + "(1,95,1212.5000000000002,1424951977653)\n" +
+					"(0,75,987.5000000000001,1424951978653)\n" + "(1,100,1240.277777777778,1424951978653)\n" +
+					"(0,80,1009.7222222222223,1424951979654)\n" + "(1,100,1268.0555555555559,1424951979654)\n" +
+					"(0,85,1033.3333333333335,1424951980654)\n" + "(1,100,1295.8333333333337,1424951980654)\n" +
+					"(0,90,1058.3333333333335,1424951981654)\n" + "(1,100,1323.6111111111115,1424951981654)\n" +
+					"(0,85,1081.9444444444446,1424951982655)\n" + "(1,100,1351.3888888888894,1424951982655)\n" +
+					"(0,90,1106.9444444444446,1424951983655)\n" + "(1,100,1379.1666666666672,1424951983655)\n" +
+					"(0,95,1133.3333333333335,1424951984655)\n" + "(1,100,1406.944444444445,1424951984656)\n" +
+					"(0,90,1158.3333333333335,1424951985656)\n" + "(1,95,1433.333333333334,1424951985656)\n" +
+					"(0,95,1184.7222222222224,1424951986656)\n" + "(1,90,1458.333333333334,1424951986656)\n" +
+					"(0,90,1209.7222222222224,1424951987656)\n" + "(1,95,1484.7222222222229,1424951987657)\n" +
+					"(0,85,1233.3333333333335,1424951988657)\n" + "(1,90,1509.7222222222229,1424951988657)\n" +
+					"(0,80,1255.5555555555557,1424951989657)\n" + "(1,95,1536.1111111111118,1424951989657)\n" +
+					"(0,85,1279.1666666666667,1424951990657)\n" + "(1,100,1563.8888888888896,1424951990658)\n" +
+					"(0,90,1304.1666666666667,1424951991658)\n" + "(1,95,1590.2777777777785,1424951991658)\n" +
+					"(0,95,1330.5555555555557,1424951992658)\n" + "(1,90,1615.2777777777785,1424951992658)\n" +
+					"(0,100,1358.3333333333335,1424951993659)\n" + "(1,95,1641.6666666666674,1424951993659)\n" +
+					"(0,100,1386.1111111111113,1424951994659)\n" + "(1,100,1669.4444444444453,1424951994659)\n" +
+					"(0,95,1412.5000000000002,1424951995659)\n" + "(1,95,1695.8333333333342,1424951995660)\n" +
+					"(0,100,1440.277777777778,1424951996660)\n" + "(1,90,1720.8333333333342,1424951996660)\n" +
+					"(0,100,1468.0555555555559,1424951997660)\n" + "(1,85,1744.4444444444453,1424951997660)\n" +
+					"(0,95,1494.4444444444448,1424951998660)\n" + "(1,80,1766.6666666666674,1424951998661)\n" +
+					"(0,100,1522.2222222222226,1424951999661)\n" + "(1,75,1787.5000000000007,1424951999661)\n" +
+					"(0,95,1548.6111111111115,1424952000661)\n" + "(1,80,1809.7222222222229,1424952000661)\n" +
+					"(0,90,1573.6111111111115,1424952001662)\n" + "(1,75,1830.555555555556,1424952001662)\n" +
+					"(0,95,1600.0000000000005,1424952002662)\n" + "(1,80,1852.7777777777783,1424952002662)\n" +
+					"(0,100,1627.7777777777783,1424952003662)\n" + "(1,85,1876.3888888888894,1424952003662)\n" +
+					"(0,100,1655.555555555556,1424952004663)\n" + "(1,80,1898.6111111111115,1424952004663)\n" +
+					"(0,95,1681.944444444445,1424952005663)\n" + "(1,85,1922.2222222222226,1424952005663)\n" +
+					"(0,100,1709.7222222222229,1424952006663)\n" + "(1,90,1947.2222222222226,1424952006664)\n" +
+					"(0,100,1737.5000000000007,1424952007664)\n" + "(1,95,1973.6111111111115,1424952007664)\n" +
+					"(0,95,1763.8888888888896,1424952008664)\n" + "(1,90,1998.6111111111115,1424952008664)\n" +
+					"(0,100,1791.6666666666674,1424952009664)\n" + "(1,85,2022.2222222222226,1424952009665)\n" +
+					"(0,95,1818.0555555555563,1424952010665)\n" + "(1,80,2044.4444444444448,1424952010665)\n" +
+					"(0,90,1843.0555555555563,1424952011665)\n" + "(1,75,2065.2777777777783,1424952011665)\n" +
+					"(0,95,1869.4444444444453,1424952012666)\n" + "(1,80,2087.5000000000005,1424952012666)\n" +
+					"(0,100,1897.222222222223,1424952013666)\n" + "(1,85,2111.1111111111118,1424952013666)\n" +
+					"(0,95,1923.611111111112,1424952014666)\n" + "(1,90,2136.1111111111118,1424952014666)\n" +
+					"(0,100,1951.3888888888898,1424952015667)\n" + "(1,85,2159.722222222223,1424952015667)\n" +
+					"(0,95,1977.7777777777787,1424952016667)\n" + "(1,90,2184.722222222223,1424952016667)\n" +
+					"(0,100,2005.5555555555566,1424952017667)\n" + "(1,95,2211.1111111111118,1424952017668)";
+
+	public static final String TOP_SPEEDS =
+			"(0,55,15.277777777777777,1424951918630)\n" +
+					"(1,50,26.38888888888889,1424951919632)\n" +
+					"(0,65,108.33333333333333,1424951924635)\n" +
+					"(1,50,26.38888888888889,1424951919632)\n" +
+					"(0,65,108.33333333333333,1424951924635)\n" +
+					"(1,65,194.4444444444444,1424951932638)\n" +
+					"(0,65,108.33333333333333,1424951924635)\n" +
+					"(1,70,213.88888888888886,1424951933638)\n" +
+					"(0,60,218.05555555555551,1424951931637)\n" +
+					"(1,75,272.2222222222222,1424951936639)\n" +
+					"(0,55,233.3333333333333,1424951932637)\n" +
+					"(1,75,272.2222222222222,1424951936639)\n" +
+					"(1,75,272.2222222222222,1424951936639)\n" +
+					"(0,55,288.88888888888886,1424951936639)\n" +
+					"(1,70,329.16666666666663,1424951939640)\n" +
+					"(0,55,373.61111111111114,1424951942641)\n" +
+					"(1,80,519.4444444444443,1424951949644)\n" +
+					"(1,85,586.111111111111,1424951952645)\n" +
+					"(0,50,487.50000000000006,1424951951644)\n" +
+					"(1,85,586.111111111111,1424951952645)\n" +
+					"(0,60,590.2777777777778,1424951959647)\n" +
+					"(1,85,586.111111111111,1424951952645)\n" +
+					"(0,75,648.6111111111112,1424951962648)\n" +
+					"(1,85,715.2777777777776,1424951958647)\n" +
+					"(1,95,858.333333333333,1424951964649)\n" +
+					"(0,80,670.8333333333334,1424951963648)\n" +
+					"(1,95,858.333333333333,1424951964649)\n" +
+					"(0,80,670.8333333333334,1424951963648)\n" +
+					"(1,100,937.4999999999998,1424951967650)\n" +
+					"(1,100,937.4999999999998,1424951967650)\n" +
+					"(0,80,670.8333333333334,1424951963648)\n" +
+					"(1,100,937.4999999999998,1424951967650)\n" +
+					"(0,85,861.1111111111112,1424951972651)\n" +
+					"(1,100,937.4999999999998,1424951967650)\n" +
+					"(1,100,937.4999999999998,1424951967650)\n" +
+					"(0,85,861.1111111111112,1424951972651)\n" +
+					"(1,100,993.0555555555554,1424951969650)\n" +
+					"(0,85,861.1111111111112,1424951972651)\n" +
+					"(1,100,1048.611111111111,1424951971651)\n" +
+					"(1,100,1130.5555555555557,1424951974652)\n" +
+					"(0,90,1058.3333333333335,1424951981654)\n" +
+					"(1,100,1158.3333333333335,1424951975652)\n" +
+					"(0,95,1133.3333333333335,1424951984655)\n" +
+					"(1,100,1240.277777777778,1424951978653)\n" +
+					"(0,95,1133.3333333333335,1424951984655)\n" +
+					"(1,100,1268.0555555555559,1424951979654)\n" +
+					"(0,95,1133.3333333333335,1424951984655)\n" +
+					"(1,100,1323.6111111111115,1424951981654)\n" +
+					"(0,95,1133.3333333333335,1424951984655)\n" +
+					"(1,100,1379.1666666666672,1424951983655)\n" +
+					"(0,100,1358.3333333333335,1424951993659)\n" +
+					"(1,100,1563.8888888888896,1424951990658)\n" +
+					"(0,100,1358.3333333333335,1424951993659)\n" +
+					"(1,100,1563.8888888888896,1424951990658)\n" +
+					"(0,100,1358.3333333333335,1424951993659)\n" +
+					"(1,100,1563.8888888888896,1424951990658)\n" +
+					"(0,100,1358.3333333333335,1424951993659)\n" +
+					"(0,100,1358.3333333333335,1424951993659)\n" +
+					"(1,100,1669.4444444444453,1424951994659)\n" +
+					"(0,100,1440.277777777778,1424951996660)\n" +
+					"(1,90,1720.8333333333342,1424951996660)\n" +
+					"(0,100,1468.0555555555559,1424951997660)\n" +
+					"(1,95,1973.6111111111115,1424952007664)\n" +
+					"(0,100,1522.2222222222226,1424951999661)\n" +
+					"(0,100,1627.7777777777783,1424952003662)\n" +
+					"(1,95,1973.6111111111115,1424952007664)\n" +
+					"(0,100,1627.7777777777783,1424952003662)\n" +
+					"(1,95,1973.6111111111115,1424952007664)\n" +
+					"(0,100,1709.7222222222229,1424952006663)\n" +
+					"(0,100,1737.5000000000007,1424952007664)\n" +
+					"(1,95,1973.6111111111115,1424952007664)\n" +
+					"(0,100,1791.6666666666674,1424952009664)\n" +
+					"(1,95,2211.1111111111118,1424952017668)\n";
+
+	public static final String TOP_CASE_CLASS_SPEEDS =
+			"CarEvent(0,55,15.277777777777777,1424951918630)\n" +
+					"CarEvent(1,50,26.38888888888889,1424951919632)\n" +
+					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
+					"CarEvent(1,50,26.38888888888889,1424951919632)\n" +
+					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
+					"CarEvent(1,65,194.4444444444444,1424951932638)\n" +
+					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
+					"CarEvent(1,70,213.88888888888886,1424951933638)\n" +
+					"CarEvent(0,60,218.05555555555551,1424951931637)\n" +
+					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
+					"CarEvent(0,55,233.3333333333333,1424951932637)\n" +
+					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
+					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
+					"CarEvent(0,55,288.88888888888886,1424951936639)\n" +
+					"CarEvent(1,70,329.16666666666663,1424951939640)\n" +
+					"CarEvent(0,55,373.61111111111114,1424951942641)\n" +
+					"CarEvent(1,80,519.4444444444443,1424951949644)\n" +
+					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
+					"CarEvent(0,50,487.50000000000006,1424951951644)\n" +
+					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
+					"CarEvent(0,60,590.2777777777778,1424951959647)\n" +
+					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
+					"CarEvent(0,75,648.6111111111112,1424951962648)\n" +
+					"CarEvent(1,85,715.2777777777776,1424951958647)\n" +
+					"CarEvent(1,95,858.333333333333,1424951964649)\n" +
+					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
+					"CarEvent(1,95,858.333333333333,1424951964649)\n" +
+					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
+					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
+					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
+					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
+					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
+					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
+					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
+					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
+					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
+					"CarEvent(1,100,993.0555555555554,1424951969650)\n" +
+					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
+					"CarEvent(1,100,1048.611111111111,1424951971651)\n" +
+					"CarEvent(1,100,1130.5555555555557,1424951974652)\n" +
+					"CarEvent(0,90,1058.3333333333335,1424951981654)\n" +
+					"CarEvent(1,100,1158.3333333333335,1424951975652)\n" +
+					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
+					"CarEvent(1,100,1240.277777777778,1424951978653)\n" +
+					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
+					"CarEvent(1,100,1268.0555555555559,1424951979654)\n" +
+					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
+					"CarEvent(1,100,1323.6111111111115,1424951981654)\n" +
+					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
+					"CarEvent(1,100,1379.1666666666672,1424951983655)\n" +
+					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
+					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
+					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
+					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
+					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
+					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
+					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
+					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
+					"CarEvent(1,100,1669.4444444444453,1424951994659)\n" +
+					"CarEvent(0,100,1440.277777777778,1424951996660)\n" +
+					"CarEvent(1,90,1720.8333333333342,1424951996660)\n" +
+					"CarEvent(0,100,1468.0555555555559,1424951997660)\n" +
+					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
+					"CarEvent(0,100,1522.2222222222226,1424951999661)\n" +
+					"CarEvent(0,100,1627.7777777777783,1424952003662)\n" +
+					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
+					"CarEvent(0,100,1627.7777777777783,1424952003662)\n" +
+					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
+					"CarEvent(0,100,1709.7222222222229,1424952006663)\n" +
+					"CarEvent(0,100,1737.5000000000007,1424952007664)\n" +
+					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
+					"CarEvent(0,100,1791.6666666666674,1424952009664)\n" +
+					"CarEvent(1,95,2211.1111111111118,1424952017668)\n";
+
+	private TopSpeedWindowingExampleData() {
+	}
+}


[17/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
deleted file mode 100644
index df3402e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/TopSpeedWindowing.java
+++ /dev/null
@@ -1,210 +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.examples.windowing;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
-import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger;
-
-import java.util.Arrays;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-/**
- * An example of grouped stream windowing where different eviction and trigger
- * policies can be used. A source fetches events from cars every 1 sec
- * containing their id, their current speed (kmh), overall elapsed distance (m)
- * and a timestamp. The streaming example triggers the top speed of each car
- * every x meters elapsed for the last y seconds.
- */
-public class TopSpeedWindowing {
-
-	private static final int NUM_CAR_EVENTS = 100;
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
-		@SuppressWarnings({"rawtypes", "serial"})
-		DataStream<Tuple4<Integer, Integer, Double, Long>> carData;
-
-		if (fileInput) {
-			carData = env.readTextFile(inputPath).map(new ParseCarData());
-		} else {
-			carData = env.addSource(CarSource.create(numOfCars));
-		}
-
-		DataStream<Tuple4<Integer, Integer, Double, Long>> topSpeeds = carData
-				.assignTimestamps(new CarTimestamp())
-				.keyBy(0)
-				.window(GlobalWindows.create())
-				.evictor(TimeEvictor.of(Time.of(evictionSec, TimeUnit.SECONDS)))
-				.trigger(DeltaTrigger.of(triggerMeters,
-						new DeltaFunction<Tuple4<Integer, Integer, Double, Long>>() {
-							private static final long serialVersionUID = 1L;
-
-							@Override
-							public double getDelta(
-									Tuple4<Integer, Integer, Double, Long> oldDataPoint,
-									Tuple4<Integer, Integer, Double, Long> newDataPoint) {
-								return newDataPoint.f2 - oldDataPoint.f2;
-							}
-						}))
-				.maxBy(1);
-
-		if (fileOutput) {
-			topSpeeds.print();
-			topSpeeds.writeAsText(outputPath);
-		} else {
-			topSpeeds.print();
-		}
-
-		env.execute("CarTopSpeedWindowingExample");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	private static class CarSource implements SourceFunction<Tuple4<Integer, Integer, Double, Long>> {
-
-		private static final long serialVersionUID = 1L;
-		private Integer[] speeds;
-		private Double[] distances;
-
-		private Random rand = new Random();
-
-		private volatile boolean isRunning = true;
-		private int counter;
-
-		private CarSource(int numOfCars) {
-			speeds = new Integer[numOfCars];
-			distances = new Double[numOfCars];
-			Arrays.fill(speeds, 50);
-			Arrays.fill(distances, 0d);
-		}
-
-		public static CarSource create(int cars) {
-			return new CarSource(cars);
-		}
-
-		@Override
-		public void run(SourceContext<Tuple4<Integer, Integer, Double, Long>> ctx) throws Exception {
-
-			while (isRunning && counter < NUM_CAR_EVENTS) {
-				Thread.sleep(100);
-				for (int carId = 0; carId < speeds.length; carId++) {
-					if (rand.nextBoolean()) {
-						speeds[carId] = Math.min(100, speeds[carId] + 5);
-					} else {
-						speeds[carId] = Math.max(0, speeds[carId] - 5);
-					}
-					distances[carId] += speeds[carId] / 3.6d;
-					Tuple4<Integer, Integer, Double, Long> record = new Tuple4<Integer, Integer, Double, Long>(carId,
-							speeds[carId], distances[carId], System.currentTimeMillis());
-					ctx.collect(record);
-					counter++;
-				}
-			}
-		}
-
-		@Override
-		public void cancel() {
-			isRunning = false;
-		}
-	}
-
-	private static class ParseCarData extends
-			RichMapFunction<String, Tuple4<Integer, Integer, Double, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple4<Integer, Integer, Double, Long> map(String record) {
-			String rawData = record.substring(1, record.length() - 1);
-			String[] data = rawData.split(",");
-			return new Tuple4<>(Integer.valueOf(data[0]), Integer.valueOf(data[1]), Double.valueOf(data[2]), Long.valueOf(data[3]));
-		}
-	}
-
-	private static class CarTimestamp implements TimestampExtractor<Tuple4<Integer, Integer, Double, Long>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple4<Integer, Integer, Double, Long> element,
-				long currentTimestamp) {
-			return element.f3;
-		}
-
-		@Override
-		public long extractWatermark(Tuple4<Integer, Integer, Double, Long> element,
-				long currentTimestamp) {
-			return element.f3 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileInput = false;
-	private static boolean fileOutput = false;
-	private static int numOfCars = 2;
-	private static int evictionSec = 10;
-	private static double triggerMeters = 50;
-	private static String inputPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			if (args.length == 2) {
-				fileInput = true;
-				fileOutput = true;
-				inputPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: TopSpeedWindowingExample <input path> <output path>");
-				return false;
-			}
-		}
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
deleted file mode 100644
index f3d57bf..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/WindowWordCount.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.windowing;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.examples.java.wordcount.util.WordCountData;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.examples.wordcount.WordCount;
-
-/**
- * Implements a windowed version of the streaming "WordCount" program.
- *
- * <p>
- * The input is a plain text file with lines separated by newline characters.
- * 
- * <p>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from
- * {@link org.apache.flink.examples.java.wordcount.util.WordCountData}.
- *
- * <p>
- * This example shows how to:
- * <ul>
- * <li>write a simple Flink Streaming program,
- * <li>use tuple data types,
- * <li>use basic windowing abstractions.
- * </ul>
- *
- */
-public class WindowWordCount {
-
-	// window parameters with default values
-	private static int windowSize = 250;
-	private static int slideSize = 150;
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		DataStream<String> text = getTextDataStream(env);
-
-		DataStream<Tuple2<String, Integer>> counts =
-		// split up the lines in pairs (2-tuples) containing: (word,1)
-		text.flatMap(new WordCount.Tokenizer())
-				// create windows of windowSize records slided every slideSize records
-				.keyBy(0)
-				.countWindow(windowSize, slideSize)
-				// group by the tuple field "0" and sum up tuple field "1"
-				.sum(1);
-
-		// emit result
-		if (fileOutput) {
-			counts.writeAsText(outputPath);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("WindowWordCount");
-	}
-
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length >= 2 && args.length <= 4) {
-				textPath = args[0];
-				outputPath = args[1];
-				if (args.length >= 3){
-					windowSize = Integer.parseInt(args[2]);
-
-					// if no slide size is specified use the
-					slideSize = args.length == 3 ? windowSize : Integer.parseInt(args[2]);
-				}
-			} else {
-				System.err.println("Usage: WindowWordCount <text path> <result path> [<window size>] [<slide size>]");
-				return false;
-			}
-		} else {
-			System.out.println("Executing WindowWordCount example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from a file.");
-			System.out.println("  Usage: WindowWordCount <text path> <result path> [<window size>] [<slide size>]");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			return env.readTextFile(textPath);
-		} else {
-			// get default test text data
-			return env.fromElements(WordCountData.WORDS);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
deleted file mode 100644
index c1a99a8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.windowing.util;
-
-public class SessionWindowingData {
-
-	public static final String EXPECTED = "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,1,3)\n" +
-			"(a,10,1)";
-
-	private SessionWindowingData() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
deleted file mode 100644
index bf63695..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/TopSpeedWindowingExampleData.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.windowing.util;
-
-public class TopSpeedWindowingExampleData {
-
-	public static final String CAR_DATA =
-					"(0,55,15.277777777777777,1424951918630)\n" + "(1,45,12.5,1424951918632)\n" +
-					"(0,50,29.166666666666664,1424951919632)\n" + "(1,50,26.38888888888889,1424951919632)\n" +
-					"(0,55,44.44444444444444,1424951920633)\n" + "(1,45,38.888888888888886,1424951920633)\n" +
-					"(0,50,58.33333333333333,1424951921634)\n" + "(1,40,50.0,1424951921634)\n" +
-					"(0,55,73.6111111111111,1424951922634)\n" + "(1,35,59.72222222222222,1424951922634)\n" +
-					"(0,60,90.27777777777777,1424951923634)\n" + "(1,40,70.83333333333333,1424951923634)\n" +
-					"(0,65,108.33333333333333,1424951924635)\n" + "(1,35,80.55555555555554,1424951924635)\n" +
-					"(0,60,125.0,1424951925635)\n" + "(1,40,91.66666666666666,1424951925635)\n" +
-					"(0,55,140.27777777777777,1424951926635)\n" + "(1,45,104.16666666666666,1424951926636)\n" +
-					"(0,60,156.94444444444443,1424951927636)\n" + "(1,50,118.05555555555554,1424951927636)\n" +
-					"(0,55,172.2222222222222,1424951928636)\n" + "(1,45,130.55555555555554,1424951928636)\n" +
-					"(0,50,186.1111111111111,1424951929636)\n" + "(1,50,144.44444444444443,1424951929637)\n" +
-					"(0,55,201.38888888888886,1424951930637)\n" + "(1,55,159.7222222222222,1424951930637)\n" +
-					"(0,60,218.05555555555551,1424951931637)\n" + "(1,60,176.38888888888886,1424951931637)\n" +
-					"(0,55,233.3333333333333,1424951932637)\n" + "(1,65,194.4444444444444,1424951932638)\n" +
-					"(0,50,247.22222222222217,1424951933638)\n" + "(1,70,213.88888888888886,1424951933638)\n" +
-					"(0,45,259.7222222222222,1424951934638)\n" + "(1,65,231.9444444444444,1424951934638)\n" +
-					"(0,50,273.6111111111111,1424951935638)\n" + "(1,70,251.38888888888886,1424951935639)\n" +
-					"(0,55,288.88888888888886,1424951936639)\n" + "(1,75,272.2222222222222,1424951936639)\n" +
-					"(0,50,302.77777777777777,1424951937639)\n" + "(1,70,291.66666666666663,1424951937639)\n" +
-					"(0,45,315.27777777777777,1424951938640)\n" + "(1,65,309.7222222222222,1424951938640)\n" +
-					"(0,50,329.1666666666667,1424951939640)\n" + "(1,70,329.16666666666663,1424951939640)\n" +
-					"(0,55,344.44444444444446,1424951940640)\n" + "(1,65,347.2222222222222,1424951940640)\n" +
-					"(0,50,358.33333333333337,1424951941641)\n" + "(1,70,366.66666666666663,1424951941641)\n" +
-					"(0,55,373.61111111111114,1424951942641)\n" + "(1,65,384.7222222222222,1424951942641)\n" +
-					"(0,50,387.50000000000006,1424951943641)\n" + "(1,70,404.16666666666663,1424951943641)\n" +
-					"(0,45,400.00000000000006,1424951944642)\n" + "(1,65,422.2222222222222,1424951944642)\n" +
-					"(0,50,413.88888888888897,1424951945642)\n" + "(1,60,438.88888888888886,1424951945642)\n" +
-					"(0,45,426.38888888888897,1424951946642)\n" + "(1,65,456.9444444444444,1424951946642)\n" +
-					"(0,40,437.50000000000006,1424951947643)\n" + "(1,70,476.38888888888886,1424951947643)\n" +
-					"(0,45,450.00000000000006,1424951948643)\n" + "(1,75,497.2222222222222,1424951948643)\n" +
-					"(0,40,461.11111111111114,1424951949643)\n" + "(1,80,519.4444444444443,1424951949644)\n" +
-					"(0,45,473.61111111111114,1424951950644)\n" + "(1,75,540.2777777777777,1424951950644)\n" +
-					"(0,50,487.50000000000006,1424951951644)\n" + "(1,80,562.4999999999999,1424951951644)\n" +
-					"(0,45,500.00000000000006,1424951952644)\n" + "(1,85,586.111111111111,1424951952645)\n" +
-					"(0,40,511.11111111111114,1424951953645)\n" + "(1,80,608.3333333333331,1424951953645)\n" +
-					"(0,35,520.8333333333334,1424951954645)\n" + "(1,75,629.1666666666665,1424951954645)\n" +
-					"(0,40,531.9444444444445,1424951955645)\n" + "(1,70,648.611111111111,1424951955646)\n" +
-					"(0,45,544.4444444444445,1424951956646)\n" + "(1,75,669.4444444444443,1424951956646)\n" +
-					"(0,50,558.3333333333334,1424951957646)\n" + "(1,80,691.6666666666665,1424951957646)\n" +
-					"(0,55,573.6111111111112,1424951958646)\n" + "(1,85,715.2777777777776,1424951958647)\n" +
-					"(0,60,590.2777777777778,1424951959647)\n" + "(1,80,737.4999999999998,1424951959647)\n" +
-					"(0,65,608.3333333333334,1424951960647)\n" + "(1,85,761.1111111111109,1424951960647)\n" +
-					"(0,70,627.7777777777778,1424951961647)\n" + "(1,80,783.333333333333,1424951961648)\n" +
-					"(0,75,648.6111111111112,1424951962648)\n" + "(1,85,806.9444444444441,1424951962648)\n" +
-					"(0,80,670.8333333333334,1424951963648)\n" + "(1,90,831.9444444444441,1424951963648)\n" +
-					"(0,75,691.6666666666667,1424951964649)\n" + "(1,95,858.333333333333,1424951964649)\n" +
-					"(0,70,711.1111111111112,1424951965649)\n" + "(1,90,883.333333333333,1424951965649)\n" +
-					"(0,75,731.9444444444446,1424951966649)\n" + "(1,95,909.722222222222,1424951966649)\n" +
-					"(0,70,751.388888888889,1424951967649)\n" + "(1,100,937.4999999999998,1424951967650)\n" +
-					"(0,75,772.2222222222224,1424951968650)\n" + "(1,100,965.2777777777776,1424951968650)\n" +
-					"(0,80,794.4444444444446,1424951969650)\n" + "(1,100,993.0555555555554,1424951969650)\n" +
-					"(0,75,815.2777777777779,1424951970651)\n" + "(1,100,1020.8333333333333,1424951970651)\n" +
-					"(0,80,837.5000000000001,1424951971651)\n" + "(1,100,1048.611111111111,1424951971651)\n" +
-					"(0,85,861.1111111111112,1424951972651)\n" + "(1,100,1076.388888888889,1424951972651)\n" +
-					"(0,80,883.3333333333334,1424951973652)\n" + "(1,95,1102.7777777777778,1424951973652)\n" +
-					"(0,75,904.1666666666667,1424951974652)\n" + "(1,100,1130.5555555555557,1424951974652)\n" +
-					"(0,70,923.6111111111112,1424951975652)\n" + "(1,100,1158.3333333333335,1424951975652)\n" +
-					"(0,75,944.4444444444446,1424951976653)\n" + "(1,100,1186.1111111111113,1424951976653)\n" +
-					"(0,80,966.6666666666667,1424951977653)\n" + "(1,95,1212.5000000000002,1424951977653)\n" +
-					"(0,75,987.5000000000001,1424951978653)\n" + "(1,100,1240.277777777778,1424951978653)\n" +
-					"(0,80,1009.7222222222223,1424951979654)\n" + "(1,100,1268.0555555555559,1424951979654)\n" +
-					"(0,85,1033.3333333333335,1424951980654)\n" + "(1,100,1295.8333333333337,1424951980654)\n" +
-					"(0,90,1058.3333333333335,1424951981654)\n" + "(1,100,1323.6111111111115,1424951981654)\n" +
-					"(0,85,1081.9444444444446,1424951982655)\n" + "(1,100,1351.3888888888894,1424951982655)\n" +
-					"(0,90,1106.9444444444446,1424951983655)\n" + "(1,100,1379.1666666666672,1424951983655)\n" +
-					"(0,95,1133.3333333333335,1424951984655)\n" + "(1,100,1406.944444444445,1424951984656)\n" +
-					"(0,90,1158.3333333333335,1424951985656)\n" + "(1,95,1433.333333333334,1424951985656)\n" +
-					"(0,95,1184.7222222222224,1424951986656)\n" + "(1,90,1458.333333333334,1424951986656)\n" +
-					"(0,90,1209.7222222222224,1424951987656)\n" + "(1,95,1484.7222222222229,1424951987657)\n" +
-					"(0,85,1233.3333333333335,1424951988657)\n" + "(1,90,1509.7222222222229,1424951988657)\n" +
-					"(0,80,1255.5555555555557,1424951989657)\n" + "(1,95,1536.1111111111118,1424951989657)\n" +
-					"(0,85,1279.1666666666667,1424951990657)\n" + "(1,100,1563.8888888888896,1424951990658)\n" +
-					"(0,90,1304.1666666666667,1424951991658)\n" + "(1,95,1590.2777777777785,1424951991658)\n" +
-					"(0,95,1330.5555555555557,1424951992658)\n" + "(1,90,1615.2777777777785,1424951992658)\n" +
-					"(0,100,1358.3333333333335,1424951993659)\n" + "(1,95,1641.6666666666674,1424951993659)\n" +
-					"(0,100,1386.1111111111113,1424951994659)\n" + "(1,100,1669.4444444444453,1424951994659)\n" +
-					"(0,95,1412.5000000000002,1424951995659)\n" + "(1,95,1695.8333333333342,1424951995660)\n" +
-					"(0,100,1440.277777777778,1424951996660)\n" + "(1,90,1720.8333333333342,1424951996660)\n" +
-					"(0,100,1468.0555555555559,1424951997660)\n" + "(1,85,1744.4444444444453,1424951997660)\n" +
-					"(0,95,1494.4444444444448,1424951998660)\n" + "(1,80,1766.6666666666674,1424951998661)\n" +
-					"(0,100,1522.2222222222226,1424951999661)\n" + "(1,75,1787.5000000000007,1424951999661)\n" +
-					"(0,95,1548.6111111111115,1424952000661)\n" + "(1,80,1809.7222222222229,1424952000661)\n" +
-					"(0,90,1573.6111111111115,1424952001662)\n" + "(1,75,1830.555555555556,1424952001662)\n" +
-					"(0,95,1600.0000000000005,1424952002662)\n" + "(1,80,1852.7777777777783,1424952002662)\n" +
-					"(0,100,1627.7777777777783,1424952003662)\n" + "(1,85,1876.3888888888894,1424952003662)\n" +
-					"(0,100,1655.555555555556,1424952004663)\n" + "(1,80,1898.6111111111115,1424952004663)\n" +
-					"(0,95,1681.944444444445,1424952005663)\n" + "(1,85,1922.2222222222226,1424952005663)\n" +
-					"(0,100,1709.7222222222229,1424952006663)\n" + "(1,90,1947.2222222222226,1424952006664)\n" +
-					"(0,100,1737.5000000000007,1424952007664)\n" + "(1,95,1973.6111111111115,1424952007664)\n" +
-					"(0,95,1763.8888888888896,1424952008664)\n" + "(1,90,1998.6111111111115,1424952008664)\n" +
-					"(0,100,1791.6666666666674,1424952009664)\n" + "(1,85,2022.2222222222226,1424952009665)\n" +
-					"(0,95,1818.0555555555563,1424952010665)\n" + "(1,80,2044.4444444444448,1424952010665)\n" +
-					"(0,90,1843.0555555555563,1424952011665)\n" + "(1,75,2065.2777777777783,1424952011665)\n" +
-					"(0,95,1869.4444444444453,1424952012666)\n" + "(1,80,2087.5000000000005,1424952012666)\n" +
-					"(0,100,1897.222222222223,1424952013666)\n" + "(1,85,2111.1111111111118,1424952013666)\n" +
-					"(0,95,1923.611111111112,1424952014666)\n" + "(1,90,2136.1111111111118,1424952014666)\n" +
-					"(0,100,1951.3888888888898,1424952015667)\n" + "(1,85,2159.722222222223,1424952015667)\n" +
-					"(0,95,1977.7777777777787,1424952016667)\n" + "(1,90,2184.722222222223,1424952016667)\n" +
-					"(0,100,2005.5555555555566,1424952017667)\n" + "(1,95,2211.1111111111118,1424952017668)";
-
-	public static final String TOP_SPEEDS =
-			"(0,55,15.277777777777777,1424951918630)\n" +
-					"(1,50,26.38888888888889,1424951919632)\n" +
-					"(0,65,108.33333333333333,1424951924635)\n" +
-					"(1,50,26.38888888888889,1424951919632)\n" +
-					"(0,65,108.33333333333333,1424951924635)\n" +
-					"(1,65,194.4444444444444,1424951932638)\n" +
-					"(0,65,108.33333333333333,1424951924635)\n" +
-					"(1,70,213.88888888888886,1424951933638)\n" +
-					"(0,60,218.05555555555551,1424951931637)\n" +
-					"(1,75,272.2222222222222,1424951936639)\n" +
-					"(0,55,233.3333333333333,1424951932637)\n" +
-					"(1,75,272.2222222222222,1424951936639)\n" +
-					"(1,75,272.2222222222222,1424951936639)\n" +
-					"(0,55,288.88888888888886,1424951936639)\n" +
-					"(1,70,329.16666666666663,1424951939640)\n" +
-					"(0,55,373.61111111111114,1424951942641)\n" +
-					"(1,80,519.4444444444443,1424951949644)\n" +
-					"(1,85,586.111111111111,1424951952645)\n" +
-					"(0,50,487.50000000000006,1424951951644)\n" +
-					"(1,85,586.111111111111,1424951952645)\n" +
-					"(0,60,590.2777777777778,1424951959647)\n" +
-					"(1,85,586.111111111111,1424951952645)\n" +
-					"(0,75,648.6111111111112,1424951962648)\n" +
-					"(1,85,715.2777777777776,1424951958647)\n" +
-					"(1,95,858.333333333333,1424951964649)\n" +
-					"(0,80,670.8333333333334,1424951963648)\n" +
-					"(1,95,858.333333333333,1424951964649)\n" +
-					"(0,80,670.8333333333334,1424951963648)\n" +
-					"(1,100,937.4999999999998,1424951967650)\n" +
-					"(1,100,937.4999999999998,1424951967650)\n" +
-					"(0,80,670.8333333333334,1424951963648)\n" +
-					"(1,100,937.4999999999998,1424951967650)\n" +
-					"(0,85,861.1111111111112,1424951972651)\n" +
-					"(1,100,937.4999999999998,1424951967650)\n" +
-					"(1,100,937.4999999999998,1424951967650)\n" +
-					"(0,85,861.1111111111112,1424951972651)\n" +
-					"(1,100,993.0555555555554,1424951969650)\n" +
-					"(0,85,861.1111111111112,1424951972651)\n" +
-					"(1,100,1048.611111111111,1424951971651)\n" +
-					"(1,100,1130.5555555555557,1424951974652)\n" +
-					"(0,90,1058.3333333333335,1424951981654)\n" +
-					"(1,100,1158.3333333333335,1424951975652)\n" +
-					"(0,95,1133.3333333333335,1424951984655)\n" +
-					"(1,100,1240.277777777778,1424951978653)\n" +
-					"(0,95,1133.3333333333335,1424951984655)\n" +
-					"(1,100,1268.0555555555559,1424951979654)\n" +
-					"(0,95,1133.3333333333335,1424951984655)\n" +
-					"(1,100,1323.6111111111115,1424951981654)\n" +
-					"(0,95,1133.3333333333335,1424951984655)\n" +
-					"(1,100,1379.1666666666672,1424951983655)\n" +
-					"(0,100,1358.3333333333335,1424951993659)\n" +
-					"(1,100,1563.8888888888896,1424951990658)\n" +
-					"(0,100,1358.3333333333335,1424951993659)\n" +
-					"(1,100,1563.8888888888896,1424951990658)\n" +
-					"(0,100,1358.3333333333335,1424951993659)\n" +
-					"(1,100,1563.8888888888896,1424951990658)\n" +
-					"(0,100,1358.3333333333335,1424951993659)\n" +
-					"(0,100,1358.3333333333335,1424951993659)\n" +
-					"(1,100,1669.4444444444453,1424951994659)\n" +
-					"(0,100,1440.277777777778,1424951996660)\n" +
-					"(1,90,1720.8333333333342,1424951996660)\n" +
-					"(0,100,1468.0555555555559,1424951997660)\n" +
-					"(1,95,1973.6111111111115,1424952007664)\n" +
-					"(0,100,1522.2222222222226,1424951999661)\n" +
-					"(0,100,1627.7777777777783,1424952003662)\n" +
-					"(1,95,1973.6111111111115,1424952007664)\n" +
-					"(0,100,1627.7777777777783,1424952003662)\n" +
-					"(1,95,1973.6111111111115,1424952007664)\n" +
-					"(0,100,1709.7222222222229,1424952006663)\n" +
-					"(0,100,1737.5000000000007,1424952007664)\n" +
-					"(1,95,1973.6111111111115,1424952007664)\n" +
-					"(0,100,1791.6666666666674,1424952009664)\n" +
-					"(1,95,2211.1111111111118,1424952017668)\n";
-
-	public static final String TOP_CASE_CLASS_SPEEDS =
-			"CarEvent(0,55,15.277777777777777,1424951918630)\n" +
-					"CarEvent(1,50,26.38888888888889,1424951919632)\n" +
-					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
-					"CarEvent(1,50,26.38888888888889,1424951919632)\n" +
-					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
-					"CarEvent(1,65,194.4444444444444,1424951932638)\n" +
-					"CarEvent(0,65,108.33333333333333,1424951924635)\n" +
-					"CarEvent(1,70,213.88888888888886,1424951933638)\n" +
-					"CarEvent(0,60,218.05555555555551,1424951931637)\n" +
-					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
-					"CarEvent(0,55,233.3333333333333,1424951932637)\n" +
-					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
-					"CarEvent(1,75,272.2222222222222,1424951936639)\n" +
-					"CarEvent(0,55,288.88888888888886,1424951936639)\n" +
-					"CarEvent(1,70,329.16666666666663,1424951939640)\n" +
-					"CarEvent(0,55,373.61111111111114,1424951942641)\n" +
-					"CarEvent(1,80,519.4444444444443,1424951949644)\n" +
-					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
-					"CarEvent(0,50,487.50000000000006,1424951951644)\n" +
-					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
-					"CarEvent(0,60,590.2777777777778,1424951959647)\n" +
-					"CarEvent(1,85,586.111111111111,1424951952645)\n" +
-					"CarEvent(0,75,648.6111111111112,1424951962648)\n" +
-					"CarEvent(1,85,715.2777777777776,1424951958647)\n" +
-					"CarEvent(1,95,858.333333333333,1424951964649)\n" +
-					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
-					"CarEvent(1,95,858.333333333333,1424951964649)\n" +
-					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
-					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
-					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
-					"CarEvent(0,80,670.8333333333334,1424951963648)\n" +
-					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
-					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
-					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
-					"CarEvent(1,100,937.4999999999998,1424951967650)\n" +
-					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
-					"CarEvent(1,100,993.0555555555554,1424951969650)\n" +
-					"CarEvent(0,85,861.1111111111112,1424951972651)\n" +
-					"CarEvent(1,100,1048.611111111111,1424951971651)\n" +
-					"CarEvent(1,100,1130.5555555555557,1424951974652)\n" +
-					"CarEvent(0,90,1058.3333333333335,1424951981654)\n" +
-					"CarEvent(1,100,1158.3333333333335,1424951975652)\n" +
-					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
-					"CarEvent(1,100,1240.277777777778,1424951978653)\n" +
-					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
-					"CarEvent(1,100,1268.0555555555559,1424951979654)\n" +
-					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
-					"CarEvent(1,100,1323.6111111111115,1424951981654)\n" +
-					"CarEvent(0,95,1133.3333333333335,1424951984655)\n" +
-					"CarEvent(1,100,1379.1666666666672,1424951983655)\n" +
-					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
-					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
-					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
-					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
-					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
-					"CarEvent(1,100,1563.8888888888896,1424951990658)\n" +
-					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
-					"CarEvent(0,100,1358.3333333333335,1424951993659)\n" +
-					"CarEvent(1,100,1669.4444444444453,1424951994659)\n" +
-					"CarEvent(0,100,1440.277777777778,1424951996660)\n" +
-					"CarEvent(1,90,1720.8333333333342,1424951996660)\n" +
-					"CarEvent(0,100,1468.0555555555559,1424951997660)\n" +
-					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
-					"CarEvent(0,100,1522.2222222222226,1424951999661)\n" +
-					"CarEvent(0,100,1627.7777777777783,1424952003662)\n" +
-					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
-					"CarEvent(0,100,1627.7777777777783,1424952003662)\n" +
-					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
-					"CarEvent(0,100,1709.7222222222229,1424952006663)\n" +
-					"CarEvent(0,100,1737.5000000000007,1424952007664)\n" +
-					"CarEvent(1,95,1973.6111111111115,1424952007664)\n" +
-					"CarEvent(0,100,1791.6666666666674,1424952009664)\n" +
-					"CarEvent(1,95,2211.1111111111118,1424952017668)\n";
-
-	private TopSpeedWindowingExampleData() {
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
deleted file mode 100644
index 591ef51..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.examples.wordcount;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.examples.java.wordcount.util.WordCountData;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.util.Collector;
-
-/**
- * This example shows an implementation of WordCount without using the Tuple2
- * type, but a custom class.
- * 
- * <p>
- * Usage: <code>WordCount &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}.
- * 
- * <p>
- * This example shows how to:
- * <ul>
- * <li>use POJO data types,
- * <li>write a simple Flink program,
- * <li>write and use user-defined functions. 
- * </ul>
- */
-public class PojoExample {
-	
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		DataStream<String> text = getTextDataStream(env);
-
-		DataStream<Word> counts =
-		// split up the lines into Word objects
-		text.flatMap(new Tokenizer())
-		// group by the field word and sum up the frequency
-				.keyBy("word").sum("frequency");
-
-		if (fileOutput) {
-			counts.writeAsText(outputPath);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("WordCount Pojo Example");
-	}
-
-	// *************************************************************************
-	// DATA TYPES
-	// *************************************************************************
-
-	/**
-	 * This is the POJO (Plain Old Java Object) that is being used for all the
-	 * operations. As long as all fields are public or have a getter/setter, the
-	 * system can handle them
-	 */
-	public static class Word {
-
-		private String word;
-		private Integer frequency;
-
-		public Word() {
-		}
-
-		public Word(String word, int i) {
-			this.word = word;
-			this.frequency = i;
-		}
-
-		public String getWord() {
-			return word;
-		}
-
-		public void setWord(String word) {
-			this.word = word;
-		}
-
-		public Integer getFrequency() {
-			return frequency;
-		}
-
-		public void setFrequency(Integer frequency) {
-			this.frequency = frequency;
-		}
-
-		@Override
-		public String toString() {
-			return "(" + word + "," + frequency + ")";
-		}
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Implements the string tokenizer that splits sentences into words as a
-	 * user-defined FlatMapFunction. The function takes a line (String) and
-	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
-	 * Integer>).
-	 */
-	public static final class Tokenizer implements FlatMapFunction<String, Word> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<Word> out) {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Word(token, 1));
-				}
-			}
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: PojoExample <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing PojoExample example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from a file.");
-			System.out.println("  Usage: PojoExample <text path> <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			return env.readTextFile(textPath);
-		} else {
-			// get default test text data
-			return env.fromElements(WordCountData.WORDS);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
deleted file mode 100644
index a594c94..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS 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.examples.wordcount;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.examples.java.wordcount.util.WordCountData;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.util.Collector;
-
-/**
- * Implements the "WordCount" program that computes a simple word occurrence
- * histogram over text files in a streaming fashion.
- * 
- * <p>
- * The input is a plain text file with lines separated by newline characters.
- * 
- * <p>
- * Usage: <code>WordCount &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}.
- * 
- * <p>
- * This example shows how to:
- * <ul>
- * <li>write a simple Flink Streaming program,
- * <li>use tuple data types,
- * <li>write and use user-defined functions.
- * </ul>
- * 
- */
-public class WordCount {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		DataStream<String> text = getTextDataStream(env);
-
-		DataStream<Tuple2<String, Integer>> counts =
-		// split up the lines in pairs (2-tuples) containing: (word,1)
-		text.flatMap(new Tokenizer())
-		// group by the tuple field "0" and sum up tuple field "1"
-				.keyBy(0).sum(1);
-
-		// emit result
-		if (fileOutput) {
-			counts.writeAsText(outputPath);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("Streaming WordCount");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Implements the string tokenizer that splits sentences into words as a
-	 * user-defined FlatMapFunction. The function takes a line (String) and
-	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
-	 * Integer>).
-	 */
-	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<Tuple2<String, Integer>> out)
-				throws Exception {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Tuple2<String, Integer>(token, 1));
-				}
-			}
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: WordCount <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing WordCount example with built-in default data.");
-			System.out.println("  Provide parameters to read input data from a file.");
-			System.out.println("  Usage: WordCount <text path> <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			return env.readTextFile(textPath);
-		} else {
-			// get default test text data
-			return env.fromElements(WordCountData.WORDS);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
deleted file mode 100644
index 42484e8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.scala.examples.join
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.streaming.api.TimeCharacteristic
-import org.apache.flink.streaming.api.scala._
-import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows
-import org.apache.flink.streaming.api.windowing.time.Time
-
-import scala.Stream._
-import scala.language.postfixOps
-import scala.util.Random
-
-object WindowJoin {
-
-  // *************************************************************************
-  // PROGRAM
-  // *************************************************************************
-
-  case class Grade(time: Long, name: String, grade: Int)
-  case class Salary(time: Long, name: String, salary: Int)
-  case class Person(name: String, grade: Int, salary: Int)
-
-  def main(args: Array[String]) {
-
-    if (!parseParameters(args)) {
-      return
-    }
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-
-    //Create streams for grades and salaries by mapping the inputs to the corresponding objects
-    val grades = setGradesInput(env)
-    val salaries = setSalariesInput(env)
-
-    //Join the two input streams by name on the last 2 seconds every second and create new
-    //Person objects containing both grade and salary
-    val joined = grades.join(salaries)
-        .where(_.name)
-        .equalTo(_.name)
-        .window(SlidingTimeWindows.of(Time.of(2, TimeUnit.SECONDS), Time.of(1, TimeUnit.SECONDS)))
-        .apply { (g, s) => Person(g.name, g.grade, s.salary) }
-
-    if (fileOutput) {
-      joined.writeAsText(outputPath)
-    } else {
-      joined.print()
-    }
-
-    env.execute("WindowJoin")
-  }
-
-  // *************************************************************************
-  // USER FUNCTIONS
-  // *************************************************************************
-
-  val names = Array("tom", "jerry", "alice", "bob", "john", "grace")
-  val gradeCount = 5
-  val salaryMax = 10000
-  val sleepInterval = 100
-  
-  def gradeStream: Stream[(Long, String, Int)] = {
-    def gradeMapper(names: Array[String])(x: Int): (Long, String, Int) =
-      {
-        if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
-        (System.currentTimeMillis(),names(Random.nextInt(names.length)),Random.nextInt(gradeCount))
-      }
-    range(1, 100).map(gradeMapper(names))
-  }
-
-  def salaryStream: Stream[(Long, String, Int)] = {
-    def salaryMapper(x: Int): (Long, String, Int) =
-      {
-        if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
-        (System.currentTimeMillis(), names(Random.nextInt(names.length)), Random.nextInt(salaryMax))
-      }
-    range(1, 100).map(salaryMapper)
-  }
-
-  def parseMap(line : String): (Long, String, Int) = {
-    val record = line.substring(1, line.length - 1).split(",")
-    (record(0).toLong, record(1), record(2).toInt)
-  }
-
-  // *************************************************************************
-  // UTIL METHODS
-  // *************************************************************************
-
-  private var fileInput: Boolean = false
-  private var fileOutput: Boolean = false
-
-  private var gradesPath: String = null
-  private var salariesPath: String = null
-  private var outputPath: String = null
-
-  private def parseParameters(args: Array[String]): Boolean = {
-    if (args.length > 0) {
-      if (args.length == 1) {
-        fileOutput = true
-        outputPath = args(0)
-      }
-      else if (args.length == 3) {
-        fileInput = true
-        fileOutput = true
-        gradesPath = args(0)
-        salariesPath = args(1)
-        outputPath = args(2)
-      } else {
-        System.err.println("Usage: WindowJoin <result path> or WindowJoin <input path 1> " +
-          "<input path 2> <result path>")
-        return false
-      }
-    } else {
-      System.out.println("Executing WindowJoin with generated data.")
-      System.out.println("  Provide parameter to write to file.")
-      System.out.println("  Usage: WindowJoin <result path>")
-    }
-    true
-  }
-
-  private def setGradesInput(env: StreamExecutionEnvironment) : DataStream[Grade] = {
-    if (fileInput) {
-      env.readTextFile(gradesPath).map(parseMap _ ).map(x => Grade(x._1, x._2, x._3))
-    } else {
-      env.fromCollection(gradeStream).map(x => Grade(x._1, x._2, x._3))
-    }
-  }
-
-  private def setSalariesInput(env: StreamExecutionEnvironment) : DataStream[Salary] = {
-    if (fileInput) {
-      env.readTextFile(salariesPath).map(parseMap _).map(x => Salary(x._1, x._2, x._3))
-    }
-    else {
-      env.fromCollection(salaryStream).map(x => Salary(x._1, x._2, x._3))
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
deleted file mode 100644
index 9ec17d4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
+++ /dev/null
@@ -1,93 +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.scala.examples.socket
-
-import org.apache.flink.streaming.api.scala._
-
-import scala.language.postfixOps
-
-/**
- * This example shows an implementation of WordCount with data from a text socket. 
- * To run the example make sure that the service providing the text data is already up and running.
- *
- * To start an example socket text stream on your local machine run netcat from a command line, 
- * where the parameter specifies the port number:
- *
- * {{{
- *   nc -lk 9999
- * }}}
- *
- * Usage:
- * {{{
- *   SocketTextStreamWordCount <hostname> <port> <output path>
- * }}}
- *
- * This example shows how to:
- *
- *   - use StreamExecutionEnvironment.socketTextStream
- *   - write a simple Flink Streaming program in scala.
- *   - write and use user-defined functions.
- */
-object SocketTextStreamWordCount {
-
-  def main(args: Array[String]) {
-    if (!parseParameters(args)) {
-      return
-    }
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    //Create streams for names and ages by mapping the inputs to the corresponding objects
-    val text = env.socketTextStream(hostName, port)
-    val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
-      .map { (_, 1) }
-      .keyBy(0)
-      .sum(1)
-
-    if (fileOutput) {
-      counts.writeAsText(outputPath, 1)
-    } else {
-      counts print
-    }
-
-    env.execute("Scala SocketTextStreamWordCount Example")
-  }
-
-  private def parseParameters(args: Array[String]): Boolean = {
-      if (args.length == 3) {
-        fileOutput = true
-        hostName = args(0)
-        port = args(1).toInt
-        outputPath = args(2)
-      } else if (args.length == 2) {
-        hostName = args(0)
-        port = args(1).toInt
-      } else {
-        System.err.println("Usage: SocketTextStreamWordCount <hostname> <port> [<output path>]")
-        return false
-      }
-    true
-  }
-
-  private var fileOutput: Boolean = false
-  private var hostName: String = null
-  private var port: Int = 0
-  private var outputPath: String = null
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
deleted file mode 100644
index f26f32c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.scala.examples.windowing
-
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.streaming.api.TimeCharacteristic
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction
-import org.apache.flink.streaming.api.scala._
-import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
-import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor
-import org.apache.flink.streaming.api.windowing.time.Time
-import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger
-
-import scala.Stream._
-import scala.math._
-import scala.language.postfixOps
-import scala.util.Random
-
-/**
- * An example of grouped stream windowing where different eviction and 
- * trigger policies can be used. A source fetches events from cars 
- * every 1 sec containing their id, their current speed (kmh),
- * overall elapsed distance (m) and a timestamp. The streaming
- * example triggers the top speed of each car every x meters elapsed 
- * for the last y seconds.
- */
-object TopSpeedWindowing {
-
-  // *************************************************************************
-  // PROGRAM
-  // *************************************************************************
-
-  case class CarEvent(carId: Int, speed: Int, distance: Double, time: Long)
-
-  val numOfCars = 2
-  val evictionSec = 10
-  val triggerMeters = 50d
-
-  def main(args: Array[String]) {
-    if (!parseParameters(args)) {
-      return
-    }
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    env.setParallelism(1)
-
-    val cars = setCarsInput(env)
-
-    val topSeed = cars
-      .assignAscendingTimestamps( _.time )
-      .keyBy("carId")
-      .window(GlobalWindows.create)
-      .evictor(TimeEvictor.of(Time.of(evictionSec * 1000, TimeUnit.MILLISECONDS)))
-      .trigger(DeltaTrigger.of(triggerMeters, new DeltaFunction[CarEvent] {
-        def getDelta(oldSp: CarEvent, newSp: CarEvent): Double = newSp.distance - oldSp.distance
-      }))
-//      .window(Time.of(evictionSec * 1000, (car : CarEvent) => car.time))
-//      .every(Delta.of[CarEvent](triggerMeters,
-//          (oldSp,newSp) => newSp.distance-oldSp.distance, CarEvent(0,0,0,0)))
-      .maxBy("speed")
-
-    if (fileOutput) {
-      topSeed.writeAsText(outputPath)
-    } else {
-      topSeed.print
-    }
-
-    env.execute("TopSpeedWindowing")
-
-  }
-
-  // *************************************************************************
-  // USER FUNCTIONS
-  // *************************************************************************
-
-  def genCarStream(): Stream[CarEvent] = {
-
-    def nextSpeed(carEvent : CarEvent) : CarEvent =
-    {
-      val next =
-        if (Random.nextBoolean) min(100, carEvent.speed + 5) else max(0, carEvent.speed - 5)
-      CarEvent(carEvent.carId, next, carEvent.distance + next/3.6d,System.currentTimeMillis)
-    }
-    def carStream(speeds : Stream[CarEvent]) : Stream[CarEvent] =
-    {
-      Thread.sleep(1000)
-      speeds.append(carStream(speeds.map(nextSpeed)))
-    }
-    carStream(range(0, numOfCars).map(CarEvent(_,50,0,System.currentTimeMillis())))
-  }
-
-  def parseMap(line : String): (Int, Int, Double, Long) = {
-    val record = line.substring(1, line.length - 1).split(",")
-    (record(0).toInt, record(1).toInt, record(2).toDouble, record(3).toLong)
-  }
-
-  // *************************************************************************
-  // UTIL METHODS
-  // *************************************************************************
-
-  var fileInput = false
-  var fileOutput = false
-  var inputPath : String = null
-  var outputPath : String = null
-
-  def parseParameters(args: Array[String]): Boolean = {
-    if (args.length > 0) {
-      if (args.length == 2) {
-        fileInput = true
-        fileOutput = true
-        inputPath = args(0)
-        outputPath = args(1)
-        true
-      } else {
-        System.err.println("Usage: TopSpeedWindowing <input path> <output path>")
-        false
-      }
-    } else {
-      true
-    }
-  }
-
-  private def setCarsInput(env: StreamExecutionEnvironment) : DataStream[CarEvent] = {
-    if (fileInput) {
-      env.readTextFile(inputPath).map(parseMap(_)).map(x => CarEvent(x._1, x._2, x._3, x._4))
-    } else {
-      env.fromCollection(genCarStream())
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java
deleted file mode 100644
index 07d6766..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleJavaPrograms.iteration;
-
-import org.apache.flink.streaming.examples.iteration.IterateExample;
-import org.apache.flink.streaming.examples.iteration.util.IterateExampleData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class IterateExampleITCase extends StreamingProgramTestBase {
-
-
-	protected String inputPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		inputPath = createTempFile("fibonacciInput.txt", IterateExampleData.INPUT_PAIRS);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(IterateExampleData.RESULTS, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		IterateExample.main(new String[]{inputPath, resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
deleted file mode 100644
index e657b67..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleJavaPrograms.join;
-
-import org.apache.flink.streaming.examples.join.WindowJoin;
-import org.apache.flink.streaming.examples.join.util.WindowJoinData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class WindowJoinITCase extends StreamingProgramTestBase {
-
-	protected String gradesPath;
-	protected String salariesPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
-		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		// since the two sides of the join might have different speed
-		// the exact output can not be checked just whether it is well-formed
-		// checks that the result lines look like e.g. (bob, 2, 2015)
-		checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java
deleted file mode 100644
index 83569dc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java
+++ /dev/null
@@ -1,42 +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.test.exampleJavaPrograms.ml;
-
-import org.apache.flink.streaming.examples.ml.IncrementalLearningSkeleton;
-import org.apache.flink.streaming.examples.ml.util.IncrementalLearningSkeletonData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class IncrementalLearningSkeletonITCase extends StreamingProgramTestBase {
-
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(IncrementalLearningSkeletonData.RESULTS, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		IncrementalLearningSkeleton.main(new String[]{resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java
deleted file mode 100644
index 838834b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java
+++ /dev/null
@@ -1,30 +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.test.exampleJavaPrograms.socket;
-
-import org.apache.flink.streaming.examples.socket.SocketTextStreamWordCount;
-import org.apache.flink.streaming.util.SocketProgramITCaseBase;
-
-public class SocketTextStreamWordCountITCase extends SocketProgramITCaseBase {
-
-	@Override
-	protected void testProgram() throws Exception {
-		SocketTextStreamWordCount.main(new String[]{HOST, port.toString(), resultPath});
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java
deleted file mode 100644
index 7850082..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java
+++ /dev/null
@@ -1,42 +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.test.exampleJavaPrograms.twitter;
-
-import org.apache.flink.streaming.examples.twitter.TwitterStream;
-import org.apache.flink.streaming.examples.twitter.util.TwitterStreamData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class TwitterStreamITCase extends StreamingProgramTestBase {
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(TwitterStreamData.STREAMING_COUNTS_AS_TUPLES, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		TwitterStream.main(new String[]{resultPath});
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java
deleted file mode 100644
index 7f46be9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java
+++ /dev/null
@@ -1,42 +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.test.exampleJavaPrograms.windowing;
-
-import org.apache.flink.streaming.examples.windowing.SessionWindowing;
-import org.apache.flink.streaming.examples.windowing.util.SessionWindowingData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class SessionWindowingITCase extends StreamingProgramTestBase {
-
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(SessionWindowingData.EXPECTED, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		SessionWindowing.main(new String[]{resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 37812c9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleJavaPrograms.windowing;
-
-import org.apache.flink.streaming.examples.windowing.TopSpeedWindowing;
-import org.apache.flink.streaming.examples.windowing.util.TopSpeedWindowingExampleData;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class TopSpeedWindowingExampleITCase extends StreamingProgramTestBase {
-	
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		setParallelism(1); //needed to ensure total ordering for windows
-		textPath = createTempFile("text.txt", TopSpeedWindowingExampleData.CAR_DATA);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(TopSpeedWindowingExampleData.TOP_SPEEDS, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		TopSpeedWindowing.main(new String[]{textPath, resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java
deleted file mode 100644
index e7cce60..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleJavaPrograms.windowing;
-
-import org.apache.flink.streaming.examples.windowing.WindowWordCount;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class WindowWordCountITCase extends StreamingProgramTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-	protected String windowSize = "250";
-	protected String slideSize = "150";
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		// since the parallel tokenizers might have different speed
-		// the exact output can not be checked just whether it is well-formed
-		// checks that the result lines look like e.g. (faust, 2)
-		checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d)+\\)");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WindowWordCount.main(new String[]{textPath, resultPath, windowSize, slideSize});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java
deleted file mode 100644
index 6e3c213..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleJavaPrograms.wordcount;
-
-import org.apache.flink.streaming.examples.wordcount.PojoExample;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class PojoExampleITCase extends StreamingProgramTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		PojoExample.main(new String[]{textPath, resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java
deleted file mode 100644
index fcf568e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.test.exampleJavaPrograms.wordcount;
-
-import org.apache.flink.streaming.examples.wordcount.WordCount;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class WordCountITCase extends StreamingProgramTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		textPath = createTempFile("text.txt", WordCountData.TEXT);
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		WordCount.main(new String[]{textPath, resultPath});
-	}
-}


[03/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
new file mode 100644
index 0000000..176a07f
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -0,0 +1,1077 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.common.functions.RichFilterFunction;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
+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.io.CsvOutputFormat;
+import org.apache.flink.api.java.io.TextOutputFormat;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.collector.selector.OutputSelector;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.sink.FileSinkFunctionByMillis;
+import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.sink.SocketClientSink;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamFilter;
+import org.apache.flink.streaming.api.operators.StreamFlatMap;
+import org.apache.flink.streaming.api.operators.StreamMap;
+import org.apache.flink.streaming.api.operators.StreamSink;
+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.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.windowing.evictors.CountEvictor;
+import org.apache.flink.streaming.api.windowing.time.AbstractTime;
+import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.streaming.runtime.operators.ExtractTimestampsOperator;
+import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.CustomPartitionerWrapper;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
+import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
+import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.util.keys.KeySelectorUtil;
+import org.apache.flink.streaming.util.serialization.SerializationSchema;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A DataStream represents a stream of elements of the same type. A DataStream
+ * can be transformed into another DataStream by applying a transformation as
+ * for example:
+ * <ul>
+ * <li>{@link DataStream#map},
+ * <li>{@link DataStream#filter}, or
+ * </ul>
+ * 
+ * @param <T> The type of the elements in this Stream
+ */
+public class DataStream<T> {
+
+	protected final StreamExecutionEnvironment environment;
+
+	protected final StreamTransformation<T> transformation;
+
+	/**
+	 * Create a new {@link DataStream} in the given execution environment with
+	 * partitioning set to forward by default.
+	 *
+	 * @param environment The StreamExecutionEnvironment
+	 */
+	public DataStream(StreamExecutionEnvironment environment, StreamTransformation<T> transformation) {
+		this.environment = Preconditions.checkNotNull(environment, "Execution Environment must not be null.");
+		this.transformation = Preconditions.checkNotNull(transformation, "Stream Transformation must not be null.");
+	}
+
+	/**
+	 * Returns the ID of the {@link DataStream} in the current {@link StreamExecutionEnvironment}.
+	 * 
+	 * @return ID of the DataStream
+	 */
+	public Integer getId() {
+		return transformation.getId();
+	}
+
+	/**
+	 * Gets the parallelism for this operator.
+	 * 
+	 * @return The parallelism set for this operator.
+	 */
+	public int getParallelism() {
+		return transformation.getParallelism();
+	}
+
+	/**
+	 * Gets the type of the stream.
+	 * 
+	 * @return The type of the datastream.
+	 */
+	public TypeInformation<T> getType() {
+		return transformation.getOutputType();
+	}
+
+	/**
+	 * Invokes the {@link org.apache.flink.api.java.ClosureCleaner}
+	 * on the given function if closure cleaning is enabled in the {@link ExecutionConfig}.
+	 *
+	 * @return The cleaned Function
+	 */
+	protected <F> F clean(F f) {
+		return getExecutionEnvironment().clean(f);
+	}
+
+	/**
+	 * Returns the {@link StreamExecutionEnvironment} that was used to create this
+	 * {@link DataStream}
+	 *
+	 * @return The Execution Environment
+	 */
+	public StreamExecutionEnvironment getExecutionEnvironment() {
+		return environment;
+	}
+
+	public ExecutionConfig getExecutionConfig() {
+		return environment.getConfig();
+	}
+
+	/**
+	 * Creates a new {@link DataStream} by merging {@link DataStream} outputs of
+	 * the same type with each other. The DataStreams merged using this operator
+	 * will be transformed simultaneously.
+	 * 
+	 * @param streams
+	 *            The DataStreams to union output with.
+	 * @return The {@link DataStream}.
+	 */
+	@SafeVarargs
+	public final DataStream<T> union(DataStream<T>... streams) {
+		List<StreamTransformation<T>> unionedTransforms = new ArrayList<>();
+		unionedTransforms.add(this.transformation);
+
+		Collection<StreamTransformation<?>> thisPredecessors = this.getTransformation().getTransitivePredecessors();
+
+		for (DataStream<T> newStream : streams) {
+			if (!(newStream.getParallelism() == this.getParallelism())) {
+				throw new UnsupportedClassVersionError(
+						"DataStream can only be unioned with DataStreams of the same parallelism. " +
+								"This Stream: " + this.getTransformation() +
+								", other stream: " + newStream.getTransformation());
+			}
+			if (!getType().equals(newStream.getType())) {
+				throw new IllegalArgumentException("Cannot union streams of different types: "
+						+ getType() + " and " + newStream.getType());
+			}
+			
+			Collection<StreamTransformation<?>> predecessors = newStream.getTransformation().getTransitivePredecessors();
+
+			if (predecessors.contains(this.transformation) || thisPredecessors.contains(newStream.getTransformation())) {
+				throw new UnsupportedOperationException("A DataStream cannot be unioned with itself");
+			}
+			unionedTransforms.add(newStream.getTransformation());
+		}
+		return new DataStream<T>(this.environment, new UnionTransformation<T>(unionedTransforms));
+	}
+
+	/**
+	 * Operator used for directing tuples to specific named outputs using an
+	 * {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}.
+	 * Calling this method on an operator creates a new {@link SplitStream}.
+	 * 
+	 * @param outputSelector
+	 *            The user defined
+	 *            {@link org.apache.flink.streaming.api.collector.selector.OutputSelector}
+	 *            for directing the tuples.
+	 * @return The {@link SplitStream}
+	 */
+	public SplitStream<T> split(OutputSelector<T> outputSelector) {
+		return new SplitStream<T>(this, clean(outputSelector));
+	}
+
+	/**
+	 * Creates a new {@link ConnectedStreams} by connecting
+	 * {@link DataStream} outputs of (possible) different types with each other.
+	 * The DataStreams connected using this operator can be used with
+	 * CoFunctions to apply joint transformations.
+	 * 
+	 * @param dataStream
+	 *            The DataStream with which this stream will be connected.
+	 * @return The {@link ConnectedStreams}.
+	 */
+	public <R> ConnectedStreams<T, R> connect(DataStream<R> dataStream) {
+		return new ConnectedStreams<T, R>(environment, this, dataStream);
+	}
+
+	/**
+	 * 
+	 * It creates a new {@link KeyedStream} that uses the provided key for partitioning
+	 * its operator states. 
+	 *
+	 * @param key
+	 *            The KeySelector to be used for extracting the key for partitioning
+	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
+	 */
+	public <K> KeyedStream<T, K> keyBy(KeySelector<T, K> key) {
+		return new KeyedStream<T, K>(this, clean(key));
+	}
+
+	/**
+	 * Partitions the operator state of a {@link DataStream} by the given key positions. 
+	 *
+	 * @param fields
+	 *            The position of the fields on which the {@link DataStream}
+	 *            will be grouped.
+	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
+	 */
+	public KeyedStream<T, Tuple> keyBy(int... fields) {
+		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
+			return keyBy(KeySelectorUtil.getSelectorForArray(fields, getType()));
+		} else {
+			return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
+		}
+	}
+
+	/**
+	 * Partitions the operator state of a {@link DataStream}using field expressions. 
+	 * A field expression is either the name of a public field or a getter method with parentheses
+	 * of the {@link DataStream}S underlying type. A dot can be used to drill
+	 * down into objects, as in {@code "field1.getInnerField2()" }.
+	 *
+	 * @param fields
+	 *            One or more field expressions on which the state of the {@link DataStream} operators will be
+	 *            partitioned.
+	 * @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
+	 **/
+	public KeyedStream<T, Tuple> keyBy(String... fields) {
+		return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
+	}
+
+	private KeyedStream<T, Tuple> keyBy(Keys<T> keys) {
+		return new KeyedStream<T, Tuple>(this, clean(KeySelectorUtil.getSelectorForKeys(keys,
+				getType(), getExecutionConfig())));
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output is
+	 * partitioned hashing on the given fields. This setting only
+	 * effects the how the outputs will be distributed between the parallel
+	 * instances of the next processing operator.
+	 *
+	 * @param fields The tuple fields that should be used for partitioning
+	 * @return The partitioned DataStream
+	 *
+	 */
+	public DataStream<T> partitionByHash(int... fields) {
+		if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
+			return partitionByHash(KeySelectorUtil.getSelectorForArray(fields, getType()));
+		} else {
+			return partitionByHash(new Keys.ExpressionKeys<T>(fields, getType()));
+		}
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output is
+	 * partitioned hashing on the given fields. This setting only
+	 * effects the how the outputs will be distributed between the parallel
+	 * instances of the next processing operator.
+	 *
+	 * @param fields The tuple fields that should be used for partitioning
+	 * @return The partitioned DataStream
+	 *
+	 */
+	public DataStream<T> partitionByHash(String... fields) {
+		return partitionByHash(new Keys.ExpressionKeys<T>(fields, getType()));
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output is
+	 * partitioned using the given {@link KeySelector}. This setting only
+	 * effects the how the outputs will be distributed between the parallel
+	 * instances of the next processing operator.
+	 *
+	 * @param keySelector The function that extracts the key from an element in the Stream
+	 * @return The partitioned DataStream
+	 */
+	public DataStream<T> partitionByHash(KeySelector<T, ?> keySelector) {
+		return setConnectionType(new HashPartitioner<T>(clean(keySelector)));
+	}
+
+	//private helper method for partitioning
+	private DataStream<T> partitionByHash(Keys<T> keys) {
+		KeySelector<T, ?> keySelector = clean(KeySelectorUtil.getSelectorForKeys(
+				keys,
+				getType(),
+				getExecutionConfig()));
+
+		return setConnectionType(new HashPartitioner<T>(keySelector));
+	}
+
+	/**
+	 * Partitions a tuple DataStream on the specified key fields using a custom partitioner.
+	 * This method takes the key position to partition on, and a partitioner that accepts the key type.
+	 * <p>
+	 * Note: This method works only on single field keys.
+	 *
+	 * @param partitioner The partitioner to assign partitions to keys.
+	 * @param field The field index on which the DataStream is to partitioned.
+	 * @return The partitioned DataStream.
+	 */
+	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, int field) {
+		Keys.ExpressionKeys<T> outExpressionKeys = new Keys.ExpressionKeys<T>(new int[]{field}, getType());
+		return partitionCustom(partitioner, outExpressionKeys);
+	}
+
+	/**
+	 * Partitions a POJO DataStream on the specified key fields using a custom partitioner.
+	 * This method takes the key expression to partition on, and a partitioner that accepts the key type.
+	 * <p>
+	 * Note: This method works only on single field keys.
+	 *
+	 * @param partitioner The partitioner to assign partitions to keys.
+	 * @param field The field index on which the DataStream is to partitioned.
+	 * @return The partitioned DataStream.
+	 */
+	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, String field) {
+		Keys.ExpressionKeys<T> outExpressionKeys = new Keys.ExpressionKeys<T>(new String[]{field}, getType());
+		return partitionCustom(partitioner, outExpressionKeys);
+	}
+
+
+	/**
+	 * Partitions a DataStream on the key returned by the selector, using a custom partitioner.
+	 * This method takes the key selector to get the key to partition on, and a partitioner that
+	 * accepts the key type.
+	 * <p>
+	 * Note: This method works only on single field keys, i.e. the selector cannot return tuples
+	 * of fields.
+	 *
+	 * @param partitioner
+	 * 		The partitioner to assign partitions to keys.
+	 * @param keySelector
+	 * 		The KeySelector with which the DataStream is partitioned.
+	 * @return The partitioned DataStream.
+	 * @see KeySelector
+	 */
+	public <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, KeySelector<T, K> keySelector) {
+		return setConnectionType(new CustomPartitionerWrapper<K, T>(clean(partitioner),
+				clean(keySelector)));
+	}
+
+	//	private helper method for custom partitioning
+	private <K> DataStream<T> partitionCustom(Partitioner<K> partitioner, Keys<T> keys) {
+		KeySelector<T, K> keySelector = KeySelectorUtil.getSelectorForOneKey(keys, partitioner, getType(), getExecutionConfig());
+
+		return setConnectionType(
+				new CustomPartitionerWrapper<K, T>(
+						clean(partitioner),
+						clean(keySelector)));
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output tuples
+	 * are broadcasted to every parallel instance of the next component.
+	 *
+	 * <p>
+	 * This setting only effects the how the outputs will be distributed between
+	 * the parallel instances of the next processing operator.
+	 * 
+	 * @return The DataStream with broadcast partitioning set.
+	 */
+	public DataStream<T> broadcast() {
+		return setConnectionType(new BroadcastPartitioner<T>());
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output tuples
+	 * are shuffled uniformly randomly to the next component.
+	 *
+	 * <p>
+	 * This setting only effects the how the outputs will be distributed between
+	 * the parallel instances of the next processing operator.
+	 * 
+	 * @return The DataStream with shuffle partitioning set.
+	 */
+	public DataStream<T> shuffle() {
+		return setConnectionType(new ShufflePartitioner<T>());
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output tuples
+	 * are forwarded to the local subtask of the next component (whenever
+	 * possible).
+	 *
+	 * <p>
+	 * This setting only effects the how the outputs will be distributed between
+	 * the parallel instances of the next processing operator.
+	 * 
+	 * @return The DataStream with forward partitioning set.
+	 */
+	public DataStream<T> forward() {
+		return setConnectionType(new ForwardPartitioner<T>());
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output tuples
+	 * are distributed evenly to instances of the next component in a Round-robin
+	 * fashion.
+	 *
+	 * <p>
+	 * This setting only effects the how the outputs will be distributed between
+	 * the parallel instances of the next processing operator.
+	 * 
+	 * @return The DataStream with rebalance partitioning set.
+	 */
+	public DataStream<T> rebalance() {
+		return setConnectionType(new RebalancePartitioner<T>());
+	}
+
+	/**
+	 * Sets the partitioning of the {@link DataStream} so that the output values
+	 * all go to the first instance of the next processing operator. Use this
+	 * setting with care since it might cause a serious performance bottleneck
+	 * in the application.
+	 * 
+	 * @return The DataStream with shuffle partitioning set.
+	 */
+	public DataStream<T> global() {
+		return setConnectionType(new GlobalPartitioner<T>());
+	}
+
+	/**
+	 * Initiates an iterative part of the program that feeds back data streams.
+	 * The iterative part needs to be closed by calling
+	 * {@link IterativeStream#closeWith(DataStream)}. The transformation of
+	 * this IterativeStream will be the iteration head. The data stream
+	 * given to the {@link IterativeStream#closeWith(DataStream)} method is
+	 * the data stream that will be fed back and used as the input for the
+	 * iteration head. The user can also use different feedback type than the
+	 * input of the iteration and treat the input and feedback streams as a
+	 * {@link ConnectedStreams} be calling
+	 * {@link IterativeStream#withFeedbackType(TypeInformation)}
+	 * <p>
+	 * A common usage pattern for streaming iterations is to use output
+	 * splitting to send a part of the closing data stream to the head. Refer to
+	 * {@link #split(OutputSelector)} for more information.
+	 * <p>
+	 * The iteration edge will be partitioned the same way as the first input of
+	 * the iteration head unless it is changed in the
+	 * {@link IterativeStream#closeWith(DataStream)} call.
+	 * <p>
+	 * By default a DataStream with iteration will never terminate, but the user
+	 * can use the maxWaitTime parameter to set a max waiting time for the
+	 * iteration head. If no data received in the set time, the stream
+	 * terminates.
+	 * 
+	 * @return The iterative data stream created.
+	 */
+	public IterativeStream<T> iterate() {
+		return new IterativeStream<T>(this, 0);
+	}
+
+	/**
+	 * Initiates an iterative part of the program that feeds back data streams.
+	 * The iterative part needs to be closed by calling
+	 * {@link IterativeStream#closeWith(DataStream)}. The transformation of
+	 * this IterativeStream will be the iteration head. The data stream
+	 * given to the {@link IterativeStream#closeWith(DataStream)} method is
+	 * the data stream that will be fed back and used as the input for the
+	 * iteration head. The user can also use different feedback type than the
+	 * input of the iteration and treat the input and feedback streams as a
+	 * {@link ConnectedStreams} be calling
+	 * {@link IterativeStream#withFeedbackType(TypeInformation)}
+	 * <p>
+	 * A common usage pattern for streaming iterations is to use output
+	 * splitting to send a part of the closing data stream to the head. Refer to
+	 * {@link #split(OutputSelector)} for more information.
+	 * <p>
+	 * The iteration edge will be partitioned the same way as the first input of
+	 * the iteration head unless it is changed in the
+	 * {@link IterativeStream#closeWith(DataStream)} call.
+	 * <p>
+	 * By default a DataStream with iteration will never terminate, but the user
+	 * can use the maxWaitTime parameter to set a max waiting time for the
+	 * iteration head. If no data received in the set time, the stream
+	 * terminates.
+	 * 
+	 * @param maxWaitTimeMillis
+	 *            Number of milliseconds to wait between inputs before shutting
+	 *            down
+	 * 
+	 * @return The iterative data stream created.
+	 */
+	public IterativeStream<T> iterate(long maxWaitTimeMillis) {
+		return new IterativeStream<T>(this, maxWaitTimeMillis);
+	}
+
+	/**
+	 * Applies a Map transformation on a {@link DataStream}. The transformation
+	 * calls a {@link MapFunction} for each element of the DataStream. Each
+	 * MapFunction call returns exactly one element. The user can also extend
+	 * {@link RichMapFunction} to gain access to other features provided by the
+	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
+	 * 
+	 * @param mapper
+	 *            The MapFunction that is called for each element of the
+	 *            DataStream.
+	 * @param <R>
+	 *            output type
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> map(MapFunction<T, R> mapper) {
+
+		TypeInformation<R> outType = TypeExtractor.getMapReturnTypes(clean(mapper), getType(),
+				Utils.getCallLocationName(), true);
+
+		return transform("Map", outType, new StreamMap<T, R>(clean(mapper)));
+	}
+
+	/**
+	 * Applies a FlatMap transformation on a {@link DataStream}. The
+	 * transformation calls a {@link FlatMapFunction} for each element of the
+	 * DataStream. Each FlatMapFunction call can return any number of elements
+	 * including none. The user can also extend {@link RichFlatMapFunction} to
+	 * gain access to other features provided by the
+	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
+	 * 
+	 * @param flatMapper
+	 *            The FlatMapFunction that is called for each element of the
+	 *            DataStream
+	 * 
+	 * @param <R>
+	 *            output type
+	 * @return The transformed {@link DataStream}.
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> flatMap(FlatMapFunction<T, R> flatMapper) {
+
+		TypeInformation<R> outType = TypeExtractor.getFlatMapReturnTypes(clean(flatMapper),
+				getType(), Utils.getCallLocationName(), true);
+
+		return transform("Flat Map", outType, new StreamFlatMap<T, R>(clean(flatMapper)));
+
+	}
+
+	/**
+	 * Applies a Filter transformation on a {@link DataStream}. The
+	 * transformation calls a {@link FilterFunction} for each element of the
+	 * DataStream and retains only those element for which the function returns
+	 * true. Elements for which the function returns false are filtered. The
+	 * user can also extend {@link RichFilterFunction} to gain access to other
+	 * features provided by the
+	 * {@link org.apache.flink.api.common.functions.RichFunction} interface.
+	 * 
+	 * @param filter
+	 *            The FilterFunction that is called for each element of the
+	 *            DataStream.
+	 * @return The filtered DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> filter(FilterFunction<T> filter) {
+		return transform("Filter", getType(), new StreamFilter<T>(clean(filter)));
+
+	}
+
+	/**
+	 * Initiates a Project transformation on a {@link Tuple} {@link DataStream}.<br/>
+	 * <b>Note: Only Tuple DataStreams can be projected.</b>
+	 *
+	 * <p>
+	 * The transformation projects each Tuple of the DataSet onto a (sub)set of
+	 * fields.
+	 * 
+	 * @param fieldIndexes
+	 *            The field indexes of the input tuples that are retained. The
+	 *            order of fields in the output tuple corresponds to the order
+	 *            of field indexes.
+	 * @return The projected DataStream
+	 * 
+	 * @see Tuple
+	 * @see DataStream
+	 */
+	public <R extends Tuple> SingleOutputStreamOperator<R, ?> project(int... fieldIndexes) {
+		return new StreamProjection<T>(this, fieldIndexes).projectTupleX();
+	}
+
+	/**
+	 * Creates a join operation. See {@link CoGroupedStreams} for an example of how the keys
+	 * and window can be specified.
+	 */
+	public <T2> CoGroupedStreams<T, T2> coGroup(DataStream<T2> otherStream) {
+		return new CoGroupedStreams<>(this, otherStream);
+	}
+
+	/**
+	 * Creates a join operation. See {@link JoinedStreams} for an example of how the keys
+	 * and window can be specified.
+	 */
+	public <T2> JoinedStreams<T, T2> join(DataStream<T2> otherStream) {
+		return new JoinedStreams<>(this, otherStream);
+	}
+
+	/**
+	 * Windows this {@code DataStream} 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
+	 *
+	 * <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).
+	 *
+	 * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)}
+	 *
+	 * @param size The size of the window.
+	 */
+	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size) {
+		return windowAll(TumblingTimeWindows.of(size));
+	}
+
+	/**
+	 * Windows this {@code DataStream} 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)}
+	 *
+	 * <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 size The size of the window.
+	 */
+	public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size, AbstractTime slide) {
+		return windowAll(SlidingTimeWindows.of(size, slide));
+	}
+
+	/**
+	 * Windows this {@code DataStream} into tumbling count windows.
+	 *
+	 * <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 size The size of the windows in number of elements.
+	 */
+	public AllWindowedStream<T, GlobalWindow> countWindowAll(long size) {
+		return windowAll(GlobalWindows.create()).trigger(PurgingTrigger.of(CountTrigger.of(size)));
+	}
+
+	/**
+	 * Windows this {@code DataStream} into sliding count windows.
+	 *
+	 * <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 size The size of the windows in number of elements.
+	 * @param slide The slide interval in number of elements.
+	 */
+	public AllWindowedStream<T, GlobalWindow> countWindowAll(long size, long slide) {
+		return windowAll(GlobalWindows.create())
+				.evictor(CountEvictor.of(size))
+				.trigger(CountTrigger.of(slide));
+	}
+
+	/**
+	 * 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> AllWindowedStream<T, W> windowAll(WindowAssigner<? super T, W> assigner) {
+		return new AllWindowedStream<>(this, assigner);
+	}
+
+	/**
+	 * Extracts a timestamp from an element and assigns it as the internal timestamp of that element.
+	 * The internal timestamps are, for example, used to to event-time window operations.
+	 *
+	 * <p>
+	 * If you know that the timestamps are strictly increasing you can use an
+	 * {@link org.apache.flink.streaming.api.functions.AscendingTimestampExtractor}. Otherwise,
+	 * you should provide a {@link TimestampExtractor} that also implements
+	 * {@link TimestampExtractor#getCurrentWatermark()} to keep track of watermarks.
+	 *
+	 * @see org.apache.flink.streaming.api.watermark.Watermark
+	 *
+	 * @param extractor The TimestampExtractor that is called for each element of the DataStream.
+	 */
+	public SingleOutputStreamOperator<T, ?> assignTimestamps(TimestampExtractor<T> extractor) {
+		// match parallelism to input, otherwise dop=1 sources could lead to some strange
+		// behaviour: the watermark will creep along very slowly because the elements
+		// from the source go to each extraction operator round robin.
+		int inputParallelism = getTransformation().getParallelism();
+		ExtractTimestampsOperator<T> operator = new ExtractTimestampsOperator<>(clean(extractor));
+		return transform("ExtractTimestamps", getTransformation().getOutputType(), operator)
+				.setParallelism(inputParallelism);
+	}
+
+	/**
+	 * Writes a DataStream to the standard output stream (stdout).
+	 *
+	 * <p>
+	 * For each element of the DataStream the result of
+	 * {@link Object#toString()} is written.
+	 * 
+	 * @return The closed DataStream.
+	 */
+	public DataStreamSink<T> print() {
+		PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>();
+		return addSink(printFunction);
+	}
+
+	/**
+	 * Writes a DataStream to the standard output stream (stderr).
+	 *
+	 * <p>
+	 * For each element of the DataStream the result of
+	 * {@link Object#toString()} is written.
+	 * 
+	 * @return The closed DataStream.
+	 */
+	public DataStreamSink<T> printToErr() {
+		PrintSinkFunction<T> printFunction = new PrintSinkFunction<T>(true);
+		return addSink(printFunction);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in text format.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
+	 * is written.
+	 * 
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * 
+	 * @return the closed DataStream.
+	 */
+	public DataStreamSink<T> writeAsText(String path) {
+		return write(new TextOutputFormat<T>(new Path(path)), 0L);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in text format. The
+	 * writing is performed periodically, in every millis milliseconds.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
+	 * is written.
+	 * 
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * @param millis
+	 *            the file update frequency
+	 * 
+	 * @return the closed DataStream
+	 */
+	public DataStreamSink<T> writeAsText(String path, long millis) {
+		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
+		return write(tof, millis);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in text format.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
+	 * is written.
+	 * 
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * @param writeMode
+	 *            Control the behavior for existing files. Options are
+	 *            NO_OVERWRITE and OVERWRITE.
+	 * 
+	 * @return the closed DataStream.
+	 */
+	public DataStreamSink<T> writeAsText(String path, WriteMode writeMode) {
+		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
+		tof.setWriteMode(writeMode);
+		return write(tof, 0L);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in text format.
+	 *
+	 * <p>
+	 * For every element of the DataStream the result of {@link Object#toString()}
+	 * is written.
+	 * 
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * @param writeMode
+	 *            Controls the behavior for existing files. Options are
+	 *            NO_OVERWRITE and OVERWRITE.
+	 * @param millis
+	 *            the file update frequency
+	 * 
+	 * @return the closed DataStream.
+	 */
+	public DataStreamSink<T> writeAsText(String path, WriteMode writeMode, long millis) {
+		TextOutputFormat<T> tof = new TextOutputFormat<T>(new Path(path));
+		tof.setWriteMode(writeMode);
+		return write(tof, millis);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in csv format.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
+	 * is written. This method can only be used on data streams of tuples.
+	 * 
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * 
+	 * @return the closed DataStream
+	 */
+	@SuppressWarnings("unchecked")
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path) {
+		Preconditions.checkArgument(getType().isTupleType(),
+				"The writeAsCsv() method can only be used on data sets of tuples.");
+		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
+				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
+		return write((OutputFormat<T>) of, 0L);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in csv format. The
+	 * writing is performed periodically, in every millis milliseconds.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
+	 * is written. This method can only be used on data streams of tuples.
+	 *
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * @param millis
+	 *            the file update frequency
+	 * 
+	 * @return the closed DataStream
+	 */
+	@SuppressWarnings("unchecked")
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, long millis) {
+		Preconditions.checkArgument(getType().isTupleType(),
+				"The writeAsCsv() method can only be used on data sets of tuples.");
+		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
+				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
+		return write((OutputFormat<T>) of, millis);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in csv format.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
+	 * is written. This method can only be used on data streams of tuples.
+	 * 
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * @param writeMode
+	 *            Controls the behavior for existing files. Options are
+	 *            NO_OVERWRITE and OVERWRITE.
+	 * 
+	 * @return the closed DataStream
+	 */
+	@SuppressWarnings("unchecked")
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, WriteMode writeMode) {
+		Preconditions.checkArgument(getType().isTupleType(),
+				"The writeAsCsv() method can only be used on data sets of tuples.");
+		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
+				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
+		if (writeMode != null) {
+			of.setWriteMode(writeMode);
+		}
+		return write((OutputFormat<T>) of, 0L);
+	}
+
+	/**
+	 * Writes a DataStream to the file specified by path in csv format. The
+	 * writing is performed periodically, in every millis milliseconds.
+	 *
+	 * <p>
+	 * For every field of an element of the DataStream the result of {@link Object#toString()}
+	 * is written. This method can only be used on data streams of tuples.
+	 * 
+	 * @param path
+	 *            the path pointing to the location the text file is written to
+	 * @param writeMode
+	 *            Controls the behavior for existing files. Options are
+	 *            NO_OVERWRITE and OVERWRITE.
+	 * @param millis
+	 *            the file update frequency
+	 * 
+	 * @return the closed DataStream
+	 */
+	@SuppressWarnings("unchecked")
+	public <X extends Tuple> DataStreamSink<T> writeAsCsv(String path, WriteMode writeMode,
+			long millis) {
+		Preconditions.checkArgument(getType().isTupleType(),
+				"The writeAsCsv() method can only be used on data sets of tuples.");
+		CsvOutputFormat<X> of = new CsvOutputFormat<X>(new Path(path),
+				CsvOutputFormat.DEFAULT_LINE_DELIMITER, CsvOutputFormat.DEFAULT_FIELD_DELIMITER);
+		if (writeMode != null) {
+			of.setWriteMode(writeMode);
+		}
+		return write((OutputFormat<T>) of, millis);
+	}
+
+	/**
+	 * Writes the DataStream to a socket as a byte array. The format of the
+	 * output is specified by a {@link SerializationSchema}.
+	 * 
+	 * @param hostName
+	 *            host of the socket
+	 * @param port
+	 *            port of the socket
+	 * @param schema
+	 *            schema for serialization
+	 * @return the closed DataStream
+	 */
+	public DataStreamSink<T> writeToSocket(String hostName, int port, SerializationSchema<T, byte[]> schema) {
+		DataStreamSink<T> returnStream = addSink(new SocketClientSink<T>(hostName, port, schema, 0));
+		returnStream.setParallelism(1); // It would not work if multiple instances would connect to the same port
+		return returnStream;
+	}
+	
+	/**
+	 * Writes the dataStream into an output, described by an OutputFormat.
+	 * 
+	 * @param format The output format
+	 * @param millis the write frequency
+	 * @return The closed DataStream
+	 */
+	public DataStreamSink<T> write(OutputFormat<T> format, long millis) {
+		return addSink(new FileSinkFunctionByMillis<T>(format, millis));
+	}
+
+	/**
+	 * Method for passing user defined operators along with the type
+	 * information that will transform the DataStream.
+	 * 
+	 * @param operatorName
+	 *            name of the operator, for logging purposes
+	 * @param outTypeInfo
+	 *            the output type of the operator
+	 * @param operator
+	 *            the object containing the transformation logic
+	 * @param <R>
+	 *            type of the return stream
+	 * @return the data stream constructed
+	 */
+	public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName, TypeInformation<R> outTypeInfo, OneInputStreamOperator<T, R> operator) {
+
+		// read the output type of the input Transform to coax out errors about MissingTypeInfo
+		transformation.getOutputType();
+
+		OneInputTransformation<T, R> resultTransform = new OneInputTransformation<T, R>(
+				this.transformation,
+				operatorName,
+				operator,
+				outTypeInfo,
+				environment.getParallelism());
+
+		@SuppressWarnings({ "unchecked", "rawtypes" })
+		SingleOutputStreamOperator<R, ?> returnStream = new SingleOutputStreamOperator(environment, resultTransform);
+
+		getExecutionEnvironment().addOperator(resultTransform);
+
+		return returnStream;
+	}
+
+	/**
+	 * Internal function for setting the partitioner for the DataStream
+	 *
+	 * @param partitioner
+	 *            Partitioner to set.
+	 * @return The modified DataStream.
+	 */
+	protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
+		return new DataStream<T>(this.getExecutionEnvironment(), new PartitionTransformation<T>(this.getTransformation(), partitioner));
+	}
+
+	/**
+	 * Adds the given sink to this DataStream. Only streams with sinks added
+	 * will be executed once the {@link StreamExecutionEnvironment#execute()}
+	 * method is called.
+	 * 
+	 * @param sinkFunction
+	 *            The object containing the sink's invoke function.
+	 * @return The closed DataStream.
+	 */
+	public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
+
+		// read the output type of the input Transform to coax out errors about MissingTypeInfo
+		transformation.getOutputType();
+
+		// configure the type if needed
+		if (sinkFunction instanceof InputTypeConfigurable) {
+			((InputTypeConfigurable) sinkFunction).setInputType(getType(), getExecutionConfig() );
+		}
+
+		StreamSink<T> sinkOperator = new StreamSink<T>(clean(sinkFunction));
+
+		DataStreamSink<T> sink = new DataStreamSink<T>(this, sinkOperator);
+
+		getExecutionEnvironment().addOperator(sink.getTransformation());
+		return sink;
+	}
+
+	/**
+	 * Returns the {@link StreamTransformation} that represents the operation that logically creates
+	 * this {@link DataStream}.
+	 *
+	 * @return The Transformation
+	 */
+	public StreamTransformation<T> getTransformation() {
+		return transformation;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
new file mode 100644
index 0000000..24104ad
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamSink;
+import org.apache.flink.streaming.api.transformations.SinkTransformation;
+
+/**
+ * A Stream Sink. This is used for emitting elements from a streaming topology.
+ *
+ * @param <T> The type of the elements in the Stream
+ */
+public class DataStreamSink<T> {
+
+	SinkTransformation<T> transformation;
+
+	@SuppressWarnings("unchecked")
+	protected DataStreamSink(DataStream<T> inputStream, StreamSink<T> operator) {
+		this.transformation = new SinkTransformation<T>(inputStream.getTransformation(), "Unnamed", operator, inputStream.getExecutionEnvironment().getParallelism());
+	}
+
+	/**
+	 * Returns the transformation that contains the actual sink operator of this sink.
+	 */
+	public SinkTransformation<T> getTransformation() {
+		return transformation;
+	}
+
+	/**
+	 * Sets the name of this sink. This name is
+	 * used by the visualization and logging during runtime.
+	 *
+	 * @return The named sink.
+	 */
+	public DataStreamSink<T> name(String name) {
+		transformation.setName(name);
+		return this;
+	}
+
+	/**
+	 * Sets the parallelism for this sink. The degree must be higher than zero.
+	 *
+	 * @param parallelism The parallelism for this sink.
+	 * @return The sink with set parallelism.
+	 */
+	public DataStreamSink<T> setParallelism(int parallelism) {
+		transformation.setParallelism(parallelism);
+		return this;
+	}
+
+	/**
+	 * Turns off chaining for this operator so thread co-location will not be
+	 * used as an optimization.
+	 *
+	 * <p>
+	 * Chaining can be turned off for the whole
+	 * job by {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#disableOperatorChaining()}
+	 * however it is not advised for performance considerations.
+	 *
+	 * @return The sink with chaining disabled
+	 */
+	public DataStreamSink<T> disableChaining() {
+		this.transformation.setChainingStrategy(ChainingStrategy.NEVER);
+		return this;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
new file mode 100644
index 0000000..d2e04a7
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.transformations.SourceTransformation;
+
+/**
+ * The DataStreamSource represents the starting point of a DataStream.
+ * 
+ * @param <T> Type of the elements in the DataStream created from the this source.
+ */
+public class DataStreamSource<T> extends SingleOutputStreamOperator<T, DataStreamSource<T>> {
+
+	boolean isParallel;
+
+	public DataStreamSource(StreamExecutionEnvironment environment,
+			TypeInformation<T> outTypeInfo, StreamSource<T> operator,
+			boolean isParallel, String sourceName) {
+		super(environment, new SourceTransformation<T>(sourceName, operator, outTypeInfo, environment.getParallelism()));
+
+		this.isParallel = isParallel;
+		if (!isParallel) {
+			setParallelism(1);
+		}
+	}
+
+	@Override
+	public DataStreamSource<T> setParallelism(int parallelism) {
+		if (parallelism > 1 && !isParallel) {
+			throw new IllegalArgumentException("Source: " + transformation.getId() + " is not a parallel source");
+		} else {
+			return (DataStreamSource<T>) super.setParallelism(parallelism);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.java
new file mode 100644
index 0000000..346bef9
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/IterativeStream.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.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.api.java.typeutils.TypeInfoParser;
+import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+
+import java.util.Collection;
+
+/**
+ * The iterative data stream represents the start of an iteration in a {@link DataStream}.
+ * 
+ * @param <T> Type of the elements in this Stream
+ */
+public class IterativeStream<T> extends SingleOutputStreamOperator<T, IterativeStream<T>> {
+
+	// We store these so that we can create a co-iteration if we need to
+	private DataStream<T> originalInput;
+	private long maxWaitTime;
+	
+	protected IterativeStream(DataStream<T> dataStream, long maxWaitTime) {
+		super(dataStream.getExecutionEnvironment(),
+				new FeedbackTransformation<T>(dataStream.getTransformation(), maxWaitTime));
+		this.originalInput = dataStream;
+		this.maxWaitTime = maxWaitTime;
+		setBufferTimeout(dataStream.environment.getBufferTimeout());
+	}
+
+	/**
+	 * Closes the iteration. This method defines the end of the iterative
+	 * program part that will be fed back to the start of the iteration.
+	 *
+	 * <p>
+	 * A common usage pattern for streaming iterations is to use output
+	 * splitting to send a part of the closing data stream to the head. Refer to
+	 * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector)}
+	 * for more information.
+	 * 
+	 * @param feedbackStream
+	 *            {@link DataStream} that will be used as input to the iteration
+	 *            head.
+	 *
+	 * @return The feedback stream.
+	 * 
+	 */
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	public DataStream<T> closeWith(DataStream<T> feedbackStream) {
+
+		Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
+
+		if (!predecessors.contains(this.transformation)) {
+			throw new UnsupportedOperationException(
+					"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
+		}
+
+		((FeedbackTransformation) getTransformation()).addFeedbackEdge(feedbackStream.getTransformation());
+
+		return feedbackStream;
+	}
+
+	/**
+	 * Changes the feedback type of the iteration and allows the user to apply
+	 * co-transformations on the input and feedback stream, as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * For type safety the user needs to define the feedback type
+	 * 
+	 * @param feedbackTypeString
+	 *            String describing the type information of the feedback stream.
+	 * @return A {@link ConnectedIterativeStreams}.
+	 */
+	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(String feedbackTypeString) {
+		return withFeedbackType(TypeInfoParser.<F> parse(feedbackTypeString));
+	}
+
+	/**
+	 * Changes the feedback type of the iteration and allows the user to apply
+	 * co-transformations on the input and feedback stream, as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * For type safety the user needs to define the feedback type
+	 * 
+	 * @param feedbackTypeClass
+	 *            Class of the elements in the feedback stream.
+	 * @return A {@link ConnectedIterativeStreams}.
+	 */
+	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(Class<F> feedbackTypeClass) {
+		return withFeedbackType(TypeExtractor.getForClass(feedbackTypeClass));
+	}
+
+	/**
+	 * Changes the feedback type of the iteration and allows the user to apply
+	 * co-transformations on the input and feedback stream, as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * For type safety the user needs to define the feedback type
+	 * 
+	 * @param feedbackType
+	 *            The type information of the feedback stream.
+	 * @return A {@link ConnectedIterativeStreams}.
+	 */
+	public <F> ConnectedIterativeStreams<T, F> withFeedbackType(TypeInformation<F> feedbackType) {
+		return new ConnectedIterativeStreams<T, F>(originalInput, feedbackType, maxWaitTime);
+	}
+	
+	/**
+	 * The {@link ConnectedIterativeStreams} represent a start of an
+	 * iterative part of a streaming program, where the original input of the
+	 * iteration and the feedback of the iteration are connected as in a
+	 * {@link ConnectedStreams}.
+	 *
+	 * <p>
+	 * The user can distinguish between the two inputs using co-transformation,
+	 * thus eliminating the need for mapping the inputs and outputs to a common
+	 * type.
+	 * 
+	 * @param <I>
+	 *            Type of the input of the iteration
+	 * @param <F>
+	 *            Type of the feedback of the iteration
+	 */
+	public static class ConnectedIterativeStreams<I, F> extends ConnectedStreams<I, F> {
+
+		private CoFeedbackTransformation<F> coFeedbackTransformation;
+
+		public ConnectedIterativeStreams(DataStream<I> input,
+				TypeInformation<F> feedbackType,
+				long waitTime) {
+			super(input.getExecutionEnvironment(),
+					input,
+					new DataStream<F>(input.getExecutionEnvironment(),
+							new CoFeedbackTransformation<F>(input.getParallelism(),
+									feedbackType,
+									waitTime)));
+			this.coFeedbackTransformation = (CoFeedbackTransformation<F>) getSecondInput().getTransformation();
+		}
+
+		/**
+		 * Closes the iteration. This method defines the end of the iterative
+		 * program part that will be fed back to the start of the iteration as
+		 * the second input in the {@link ConnectedStreams}.
+		 * 
+		 * @param feedbackStream
+		 *            {@link DataStream} that will be used as second input to
+		 *            the iteration head.
+		 * @return The feedback stream.
+		 * 
+		 */
+		@SuppressWarnings({ "rawtypes", "unchecked" })
+		public DataStream<F> closeWith(DataStream<F> feedbackStream) {
+
+			Collection<StreamTransformation<?>> predecessors = feedbackStream.getTransformation().getTransitivePredecessors();
+
+			if (!predecessors.contains(this.coFeedbackTransformation)) {
+				throw new UnsupportedOperationException(
+						"Cannot close an iteration with a feedback DataStream that does not originate from said iteration.");
+			}
+
+			coFeedbackTransformation.addFeedbackEdge(feedbackStream.getTransformation());
+
+			return feedbackStream;
+		}
+		
+		private UnsupportedOperationException groupingException = new UnsupportedOperationException(
+				"Cannot change the input partitioning of an iteration head directly. Apply the partitioning on the input and feedback streams instead.");
+		
+		@Override
+		public ConnectedStreams<I, F> keyBy(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> keyBy(String field1, String field2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> keyBy(String[] fields1, String[] fields2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> keyBy(KeySelector<I, ?> keySelector1,KeySelector<F, ?> keySelector2) {throw groupingException;}
+
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(int keyPosition1, int keyPosition2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(int[] keyPositions1, int[] keyPositions2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(String field1, String field2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(String[] fields1, String[] fields2) {throw groupingException;}
+		
+		@Override
+		public ConnectedStreams<I, F> partitionByHash(KeySelector<I, ?> keySelector1, KeySelector<F, ?> keySelector2) {throw groupingException;}
+		
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
new file mode 100644
index 0000000..cff9355
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
@@ -0,0 +1,330 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.datastream;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion;
+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.util.Collector;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ *{@code JoinedStreams} represents two {@link DataStream DataStreams} that have been joined.
+ * A streaming join operation is evaluated over elements in a window.
+ *
+ * <p>
+ * To finalize the join operation you also need to specify a {@link KeySelector} for
+ * both the first and second input and a {@link WindowAssigner}.
+ *
+ * <p>
+ * Note: Right now, the the join is being evaluated in memory so you need to ensure that the number
+ * of elements per key does not get too high. Otherwise the JVM might crash.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> one = ...;
+ * DataStream<Tuple2<String, Integer>> twp = ...;
+ *
+ * DataStream<T> result = one.join(two)
+ *     .where(new MyFirstKeySelector())
+ *     .equalTo(new MyFirstKeySelector())
+ *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ *     .apply(new MyJoinFunction());
+ * } </pre>
+ */
+public class JoinedStreams<T1, T2> {
+
+	/** The first input stream */
+	private final DataStream<T1> input1;
+
+	/** The second input stream */
+	private final DataStream<T2> input2;
+
+	/**
+	 * Creates new JoinedStreams data streams, which are the first step towards building a streaming co-group.
+	 *
+	 * @param input1 The first data stream.
+	 * @param input2 The second data stream.
+	 */
+	public JoinedStreams(DataStream<T1> input1, DataStream<T2> input2) {
+		this.input1 = requireNonNull(input1);
+		this.input2 = requireNonNull(input2);
+	}
+
+	/**
+	 * Specifies a {@link KeySelector} for elements from the first input.
+	 */
+	public <KEY> Where<KEY> where(KeySelector<T1, KEY> keySelector)  {
+		TypeInformation<KEY> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+		return new Where<>(input1.clean(keySelector), keyType);
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * CoGrouped streams that have the key for one side defined.
+	 *
+	 * @param <KEY> The type of the key.
+	 */
+	public class Where<KEY> {
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final TypeInformation<KEY> keyType;
+
+		Where(KeySelector<T1, KEY> keySelector1, TypeInformation<KEY> keyType) {
+			this.keySelector1 = keySelector1;
+			this.keyType = keyType;
+		}
+
+		/**
+		 * Specifies a {@link KeySelector} for elements from the second input.
+		 */
+		public EqualTo equalTo(KeySelector<T2, KEY> keySelector)  {
+			TypeInformation<KEY> otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+			if (!otherKey.equals(this.keyType)) {
+				throw new IllegalArgumentException("The keys for the two inputs are not equal: " +
+						"first key = " + this.keyType + " , second key = " + otherKey);
+			}
+
+			return new EqualTo(input2.clean(keySelector));
+		}
+
+		// --------------------------------------------------------------------
+
+		/**
+		 * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs.
+		 */
+		public class EqualTo {
+
+			private final KeySelector<T2, KEY> keySelector2;
+
+			EqualTo(KeySelector<T2, KEY> keySelector2) {
+				this.keySelector2 = requireNonNull(keySelector2);
+			}
+
+			/**
+			 * Specifies the window on which the co-group operation works.
+			 */
+			public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+				return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null);
+			}
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A join operation that has {@link KeySelector KeySelectors} defined for both inputs as
+	 * well as a {@link WindowAssigner}.
+	 *
+	 * @param <T1> Type of the elements from the first input
+	 * @param <T2> Type of the elements from the second input
+	 * @param <KEY> Type of the key. This must be the same for both inputs
+	 * @param <W> Type of {@link Window} on which the join operation works.
+	 */
+	public static class WithWindow<T1, T2, KEY, W extends Window> {
+		
+		private final DataStream<T1> input1;
+		private final DataStream<T2> input2;
+
+		private final KeySelector<T1, KEY> keySelector1;
+		private final KeySelector<T2, KEY> keySelector2;
+		private final TypeInformation<KEY> keyType;
+
+		private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
+
+		private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
+
+		private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
+
+		protected WithWindow(DataStream<T1> input1,
+				DataStream<T2> input2,
+				KeySelector<T1, KEY> keySelector1,
+				KeySelector<T2, KEY> keySelector2,
+				TypeInformation<KEY> keyType,
+				WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
+				Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
+				Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
+			
+			this.input1 = requireNonNull(input1);
+			this.input2 = requireNonNull(input2);
+
+			this.keySelector1 = requireNonNull(keySelector1);
+			this.keySelector2 = requireNonNull(keySelector2);
+			this.keyType = requireNonNull(keyType);
+			
+			this.windowAssigner = requireNonNull(windowAssigner);
+			
+			this.trigger = trigger;
+			this.evictor = evictor;
+		}
+
+		/**
+		 * Sets the {@code Trigger} that should be used to trigger window emission.
+		 */
+		public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+					windowAssigner, newTrigger, evictor);
+		}
+
+		/**
+		 * 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 WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
+			return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+					windowAssigner, trigger, newEvictor);
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function) {
+			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+					function,
+					JoinFunction.class,
+					true,
+					true,
+					input1.getType(),
+					input2.getType(),
+					"Join",
+					false);
+
+			return apply(function, resultType);
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+			//clean the closure
+			function = input1.getExecutionEnvironment().clean(function);
+
+			return input1.coGroup(input2)
+					.where(keySelector1)
+					.equalTo(keySelector2)
+					.window(windowAssigner)
+					.trigger(trigger)
+					.evictor(evictor)
+					.apply(new FlatJoinCoGroupFunction<>(function), resultType);
+
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function) {
+			TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+					function,
+					JoinFunction.class,
+					true,
+					true,
+					input1.getType(),
+					input2.getType(),
+					"Join",
+					false);
+
+			return apply(function, resultType);
+		}
+
+		/**
+		 * Completes the join operation with the user function that is executed
+		 * for each combination of elements with the same key in a window.
+		 */
+		public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+			//clean the closure
+			function = input1.getExecutionEnvironment().clean(function);
+
+			return input1.coGroup(input2)
+					.where(keySelector1)
+					.equalTo(keySelector2)
+					.window(windowAssigner)
+					.trigger(trigger)
+					.evictor(evictor)
+					.apply(new JoinCoGroupFunction<>(function), resultType);
+
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Implementation of the functions
+	// ------------------------------------------------------------------------
+
+	/**
+	 * CoGroup function that does a nested-loop join to get the join result.
+	 */
+	private static class JoinCoGroupFunction<T1, T2, T>
+			extends WrappingFunction<JoinFunction<T1, T2, T>>
+			implements CoGroupFunction<T1, T2, T> {
+		private static final long serialVersionUID = 1L;
+
+		public JoinCoGroupFunction(JoinFunction<T1, T2, T> wrappedFunction) {
+			super(wrappedFunction);
+		}
+
+		@Override
+		public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
+			for (T1 val1: first) {
+				for (T2 val2: second) {
+					out.collect(wrappedFunction.join(val1, val2));
+				}
+			}
+		}
+	}
+
+	/**
+	 * CoGroup function that does a nested-loop join to get the join result. (FlatJoin version)
+	 */
+	private static class FlatJoinCoGroupFunction<T1, T2, T>
+			extends WrappingFunction<FlatJoinFunction<T1, T2, T>>
+			implements CoGroupFunction<T1, T2, T> {
+		private static final long serialVersionUID = 1L;
+
+		public FlatJoinCoGroupFunction(FlatJoinFunction<T1, T2, T> wrappedFunction) {
+			super(wrappedFunction);
+		}
+
+		@Override
+		public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
+			for (T1 val1: first) {
+				for (T2 val2: second) {
+					wrappedFunction.join(val1, val2, out);
+				}
+			}
+		}
+	}
+
+}


[38/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ParallelSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ParallelSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ParallelSourceFunction.java
deleted file mode 100644
index e9a739f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/ParallelSourceFunction.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-/**
- * A stream data source that is executed in parallel. Upon execution, the runtime will
- * execute as many parallel instances of this function function as configured parallelism
- * of the source.
- *
- * <p>This interface acts only as a marker to tell the system that this source may
- * be executed in parallel. When different parallel instances are required to perform
- * different tasks, use the {@link RichParallelSourceFunction} to get access to the runtime
- * context, which reveals information like the number of parallel tasks, and which parallel
- * task the current instance is.
- *
- * @param <OUT> The type of the records produced by this source.
- */
-public interface ParallelSourceFunction<OUT> extends SourceFunction<OUT> {
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichEventTimeSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichEventTimeSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichEventTimeSourceFunction.java
deleted file mode 100644
index 6e0086d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichEventTimeSourceFunction.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-
-/**
- * Base class for implementing a parallel event-time data source that has access to context information
- * (via {@link #getRuntimeContext()}) and additional life-cycle methods
- * ({@link #open(org.apache.flink.configuration.Configuration)} and {@link #close()}.
- *
- * <p>
- * This class is useful when implementing parallel sources where different parallel subtasks
- * need to perform different work. Typical patterns for that are:
- * <ul>
- *     <li>Use {@link #getRuntimeContext()} to obtain the runtime context.</li>
- *     <li>Use {@link org.apache.flink.api.common.functions.RuntimeContext#getNumberOfParallelSubtasks()}
- *         to determine the current parallelism. It is strongly encouraged to use this method, rather than
- *         hard-wiring the parallelism, because the configured parallelism may change depending on
- *         program configuration. The parallelism may also change after recovering failures, when fewer than
- *         desired parallel worker as available.</li>
- *     <li>Use {@link org.apache.flink.api.common.functions.RuntimeContext#getIndexOfThisSubtask()} to
- *         determine which subtask the current instance of the function executes.</li>
- * </ul>
- *
- *
- * @param <OUT> The type of the records produced by this source.
- */
-public abstract class RichEventTimeSourceFunction<OUT> extends AbstractRichFunction implements EventTimeSourceFunction<OUT> {
-
-	private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java
deleted file mode 100644
index 7cbf674..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-
-/**
- * Base class for implementing a parallel data source. Upon execution, the runtime will
- * execute as many parallel instances of this function function as configured parallelism
- * of the source.
- * 
- * <p>The data source has access to context information (such as the number of parallel
- * instances of the source, and which parallel instance the current instance is)
- * via {@link #getRuntimeContext()}. It also provides additional life-cycle methods
- * ({@link #open(org.apache.flink.configuration.Configuration)} and {@link #close()}.</p>
- *
- * @param <OUT> The type of the records produced by this source.
- */
-public abstract class RichParallelSourceFunction<OUT> extends AbstractRichFunction
-		implements ParallelSourceFunction<OUT> {
-
-	private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java
deleted file mode 100644
index dd08b2a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-
-/**
- * Base class for implementing a parallel data source that has access to context information
- * (via {@link #getRuntimeContext()}) and additional life-cycle methods
- * ({@link #open(org.apache.flink.configuration.Configuration)} and {@link #close()}.
- *
- * <p>This class is useful when implementing parallel sources where different parallel subtasks
- * need to perform different work. Typical patterns for that are:
- * <ul>
- *     <li>Use {@link #getRuntimeContext()} to obtain the runtime context.</li>
- *     <li>Use {@link org.apache.flink.api.common.functions.RuntimeContext#getNumberOfParallelSubtasks()}
- *         to determine the current parallelism. It is strongly encouraged to use this method, rather than
- *         hard-wiring the parallelism, because the configured parallelism may change depending on
- *         program configuration. The parallelism may also change after recovering failures, when fewer than
- *         desired parallel worker as available.</li>
- *     <li>Use {@link org.apache.flink.api.common.functions.RuntimeContext#getIndexOfThisSubtask()} to
- *         determine which subtask the current instance of the function executes.</li>
- * </ul>
- * </p>
- *
- * @param <OUT> The type of the records produced by this source.
- */
-public abstract class RichSourceFunction<OUT> extends AbstractRichFunction implements SourceFunction<OUT> {
-
-	private static final long serialVersionUID = 1L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java
deleted file mode 100644
index 9310b71..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunction.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.runtime.util.IOUtils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.InputStreamReader;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * A source function that reads strings from a socket. The source will read bytes from the socket stream
- * and convert them to characters, each byte individually. When the delimiter character is received,
- * the function will output the current string, and begin a new string.
- * <p>
- * The function strips trailing <i>carriage return</i> characters (\r) when the delimiter is the
- * newline character (\n).
- * <p>
- * The function can be set to reconnect to the server socket in case that the stream is closed on the server side.
- */
-public class SocketTextStreamFunction implements SourceFunction<String> {
-
-	private static final long serialVersionUID = 1L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(SocketTextStreamFunction.class);
-
-	/** Default delay between successive connection attempts */
-	private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
-
-	/** Default connection timeout when connecting to the server socket (infinite) */
-	private static final int CONNECTION_TIMEOUT_TIME = 0;
-	
-	
-	private final String hostname;
-	private final int port;
-	private final char delimiter;
-	private final long maxNumRetries;
-	private final long delayBetweenRetries;
-	
-	private transient Socket currentSocket;
-	
-	private volatile boolean isRunning = true;
-
-	
-	public SocketTextStreamFunction(String hostname, int port, char delimiter, long maxNumRetries) {
-		this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
-	}
-	
-	public SocketTextStreamFunction(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) {
-		checkArgument(port > 0 && port < 65536, "port is out of range");
-		checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
-		checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
-		
-		this.hostname = checkNotNull(hostname, "hostname must not be null");
-		this.port = port;
-		this.delimiter = delimiter;
-		this.maxNumRetries = maxNumRetries;
-		this.delayBetweenRetries = delayBetweenRetries;
-	}
-
-	@Override
-	public void run(SourceContext<String> ctx) throws Exception {
-		final StringBuilder buffer = new StringBuilder();
-		long attempt = 0;
-		
-		while (isRunning) {
-			
-			try (Socket socket = new Socket()) {
-				currentSocket = socket;
-				
-				LOG.info("Connecting to server socket " + hostname + ':' + port);
-				socket.connect(new InetSocketAddress(hostname, port), CONNECTION_TIMEOUT_TIME);
-				BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()));
-
-				int data;
-				while (isRunning && (data = reader.read()) != -1) {
-					// check if the string is complete
-					if (data != delimiter) {
-						buffer.append((char) data);
-					}
-					else {
-						// truncate trailing carriage return
-						if (delimiter == '\n' && buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
-							buffer.setLength(buffer.length() - 1);
-						}
-						ctx.collect(buffer.toString());
-						buffer.setLength(0);
-					}
-				}
-			}
-
-			// if we dropped out of this loop due to an EOF, sleep and retry
-			if (isRunning) {
-				attempt++;
-				if (maxNumRetries == -1 || attempt < maxNumRetries) {
-					LOG.warn("Lost connection to server socket. Retrying in " + delayBetweenRetries + " msecs...");
-					Thread.sleep(delayBetweenRetries);
-				}
-				else {
-					// this should probably be here, but some examples expect simple exists of the stream source
-					// throw new EOFException("Reached end of stream and reconnects are not enabled.");
-					break;
-				}
-			}
-		}
-
-		// collect trailing data
-		if (buffer.length() > 0) {
-			ctx.collect(buffer.toString());
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-		
-		// we need to close the socket as well, because the Thread.interrupt() function will
-		// not wake the thread in the socketStream.read() method when blocked.
-		Socket theSocket = this.currentSocket;
-		if (theSocket != null) {
-			IOUtils.closeSocket(theSocket);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java
deleted file mode 100644
index 886d6e7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.functions.Function;
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-import java.io.Serializable;
-
-/**
- * Base interface for all stream data sources in Flink. The contract of a stream source
- * is the following: When the source should start emitting elements the {@link #run} method
- * is called with a {@link org.apache.flink.util.Collector} that can be used for emitting elements.
- * The run method can run for as long as necessary. The source must, however, react to an
- * invocation of {@link #cancel} by breaking out of its main loop.
- *
- * <p>
- * <b>Note about checkpointed sources</b> <br>
- *
- * Sources that also implement the {@link org.apache.flink.streaming.api.checkpoint.Checkpointed}
- * interface must ensure that state checkpointing, updating of internal state and emission of
- * elements are not done concurrently. This is achieved by using the provided checkpointing lock
- * object to protect update of state and emission of elements in a synchronized block.
- * </p>
- *
- * <p>
- * This is the basic pattern one should follow when implementing a (checkpointed) source:
- * </p>
- *
- * {@code
- *  public class ExampleSource<T> implements SourceFunction<T>, Checkpointed<Long> {
- *      private long count = 0L;
- *      private volatile boolean isRunning = true;
- *
- *      @Override
- *      public void run(SourceContext<T> ctx) {
- *          while (isRunning && count < 1000) {
- *              synchronized (ctx.getCheckpointLock()) {
- *                  ctx.collect(count);
- *                  count++;
- *              }
- *          }
- *      }
- *
- *      @Override
- *      public void cancel() {
- *          isRunning = false;
- *      }
- *
- *      @Override
- *      public Long snapshotState(long checkpointId, long checkpointTimestamp) { return count; }
- *
- *      @Override
- *      public void restoreState(Long state) { this.count = state; }
- * }
- * </pre>
- *
- *
- * <p>
- * <b>Note about element timestamps and watermarks:</b> <br>
- * Sources must only manually emit watermarks when they implement
- * {@link EventTimeSourceFunction }.
- * Otherwise, elements automatically get the current timestamp assigned at ingress
- * and the system automatically emits watermarks.
- *
- * @param <T> The type of the elements produced by this source.
- */
-public interface SourceFunction<T> extends Function, Serializable {
-
-	/**
-	 * Starts the source. You can use the {@link org.apache.flink.util.Collector} parameter to emit
-	 * elements. Sources that implement
-	 * {@link org.apache.flink.streaming.api.checkpoint.Checkpointed} must lock on the
-	 * checkpoint lock (using a synchronized block) before updating internal state and/or emitting
-	 * elements. Also, the update of state and emission of elements must happen in the same
-	 * synchronized block.
-	 *
-	 * @param ctx The context for interaction with the outside world.
-	 */
-	void run(SourceContext<T> ctx) throws Exception;
-
-	/**
-	 * Cancels the source. Most sources will have a while loop inside the
-	 * {@link #run} method. You need to ensure that the source will break out of this loop. This
-	 * can be achieved by having a volatile field "isRunning" that is checked in the loop and that
-	 * is set to false in this method.
-	 */
-	void cancel();
-
-	/**
-	 * Interface that source functions use to communicate with the outside world. Normally
-	 * sources would just emit elements in a loop using {@link #collect}. If the source is a
-	 * {@link org.apache.flink.streaming.api.checkpoint.Checkpointed} source it must retrieve
-	 * the checkpoint lock object and use it to protect state updates and element emission as
-	 * described in {@link org.apache.flink.streaming.api.functions.source.SourceFunction}.
-	 *
-	 * @param <T> The type of the elements produced by the source.
-	 */
-	public static interface SourceContext<T> {
-
-		/**
-		 * Emits one element from the source. The result of {@link System#currentTimeMillis()} is set as
-		 * the timestamp of the emitted element.
-		 *
-		 * @param element The element to emit
-		 */
-		void collect(T element);
-
-		/**
-		 * Emits one element from the source with the given timestamp.
-		 *
-		 * @param element The element to emit
-		 * @param timestamp The timestamp in milliseconds
-		 */
-		public void collectWithTimestamp(T element, long timestamp);
-
-		/**
-		 * Emits the given {@link org.apache.flink.streaming.api.watermark.Watermark}.
-		 *
-		 * <p>
-		 * <b>Important:</b>
-		 * Sources must only manually emit watermarks when they implement
-		 * {@link EventTimeSourceFunction}.
-		 * Otherwise, elements automatically get the current timestamp assigned at ingress
-		 * and the system automatically emits watermarks.
-		 *
-		 * @param mark The {@link Watermark} to emit
-		 */
-		void emitWatermark(Watermark mark);
-
-
-		/**
-		 * Returns the checkpoint lock. Please refer to the explanation about checkpointed sources
-		 * in {@link org.apache.flink.streaming.api.functions.source.SourceFunction}.
-		 * 
-		 * @return The object to use as the lock. 
-		 */
-		Object getCheckpointLock();
-
-		/**
-		 * This must be called when closing the source operator to allow the {@link SourceContext}
-		 * to clean up internal state.
-		 */
-		void close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java
deleted file mode 100644
index 14badf1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-
-/**
- * A stateful streaming source that emits each number from a given interval exactly once,
- * possibly in parallel.
- */
-public class StatefulSequenceSource extends RichParallelSourceFunction<Long> implements Checkpointed<Long> {
-	
-	private static final long serialVersionUID = 1L;
-
-	private final long start;
-	private final long end;
-
-	private long collected;
-
-	private volatile boolean isRunning = true;
-
-	/**
-	 * Creates a source that emits all numbers from the given interval exactly once.
-	 *
-	 * @param start Start of the range of numbers to emit.
-	 * @param end End of the range of numbers to emit.
-	 */
-	public StatefulSequenceSource(long start, long end) {
-		this.start = start;
-		this.end = end;
-	}
-
-	@Override
-	public void run(SourceContext<Long> ctx) throws Exception {
-		final Object checkpointLock = ctx.getCheckpointLock();
-
-		RuntimeContext context = getRuntimeContext();
-
-		final long stepSize = context.getNumberOfParallelSubtasks();
-		final long congruence = start + context.getIndexOfThisSubtask();
-
-		final long toCollect =
-				((end - start + 1) % stepSize > (congruence - start)) ?
-					((end - start + 1) / stepSize + 1) :
-					((end - start + 1) / stepSize);
-		
-
-		while (isRunning && collected < toCollect) {
-			synchronized (checkpointLock) {
-				ctx.collect(collected * stepSize + congruence);
-				collected++;
-			}
-		}
-	}
-
-	@Override
-	public void cancel() {
-		isRunning = false;
-	}
-
-	@Override
-	public Long snapshotState(long checkpointId, long checkpointTimestamp) {
-		return collected;
-	}
-
-	@Override
-	public void restoreState(Long state) {
-		collected = state;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.java
deleted file mode 100644
index 1d54436..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/AllWindowFunction.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.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 AllWindowFunction<IN, OUT,  W extends Window> extends Function, Serializable {
-
-	/**
-	 * Evaluates the window and outputs none or several elements.
-	 *
-	 * @param window The window that is being evaluated.
-	 * @param values The elements in the window being evaluated.
-	 * @param out A collector for emitting elements.
-	 *
-	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery.
-	 */
-	void apply(W window, Iterable<IN> values, Collector<OUT> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
deleted file mode 100644
index 69f24fe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.functions.windowing;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.operators.translation.WrappingFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-public class FoldAllWindowFunction<W extends Window, T, R>
-		extends WrappingFunction<FoldFunction<T, R>>
-		implements AllWindowFunction<T, R, W>, OutputTypeConfigurable<R> {
-	private static final long serialVersionUID = 1L;
-
-	private byte[] serializedInitialValue;
-	private TypeSerializer<R> outSerializer;
-	private transient R initialValue;
-
-	public FoldAllWindowFunction(R initialValue, FoldFunction<T, R> reduceFunction) {
-		super(reduceFunction);
-		this.initialValue = initialValue;
-	}
-
-	@Override
-	public void open(Configuration configuration) throws Exception {
-		super.open(configuration);
-
-		if (serializedInitialValue == null) {
-			throw new RuntimeException("No initial value was serialized for the fold " +
-					"window function. Probably the setOutputType method was not called.");
-		}
-
-		ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
-		InputViewDataInputStreamWrapper in = new InputViewDataInputStreamWrapper(
-				new DataInputStream(bais)
-		);
-		initialValue = outSerializer.deserialize(in);
-	}
-
-	@Override
-	public void apply(W window, Iterable<T> values, Collector<R> out) throws Exception {
-		R result = outSerializer.copy(initialValue);
-
-		for (T val: values) {
-			result = wrappedFunction.fold(result, val);
-		}
-
-		out.collect(result);
-	}
-
-	@Override
-	public void setOutputType(TypeInformation<R> outTypeInfo, ExecutionConfig executionConfig) {
-		outSerializer = outTypeInfo.createSerializer(executionConfig);
-
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		OutputViewDataOutputStreamWrapper out = new OutputViewDataOutputStreamWrapper(
-				new DataOutputStream(baos)
-		);
-
-		try {
-			outSerializer.serialize(initialValue, out);
-		} catch (IOException ioe) {
-			throw new RuntimeException("Unable to serialize initial value of type " +
-					initialValue.getClass().getSimpleName() + " of fold window function.", ioe);
-		}
-
-		serializedInitialValue = baos.toByteArray();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
deleted file mode 100644
index 04d2ac7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.functions.windowing;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.operators.translation.WrappingFunction;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-public class FoldWindowFunction<K, W extends Window, T, R>
-		extends WrappingFunction<FoldFunction<T, R>>
-		implements WindowFunction<T, R, K, W>, OutputTypeConfigurable<R> {
-	private static final long serialVersionUID = 1L;
-
-	private byte[] serializedInitialValue;
-	private TypeSerializer<R> outSerializer;
-	private transient R initialValue;
-
-	public FoldWindowFunction(R initialValue, FoldFunction<T, R> reduceFunction) {
-		super(reduceFunction);
-		this.initialValue = initialValue;
-	}
-
-	@Override
-	public void open(Configuration configuration) throws Exception {
-		super.open(configuration);
-
-		if (serializedInitialValue == null) {
-			throw new RuntimeException("No initial value was serialized for the fold " +
-					"window function. Probably the setOutputType method was not called.");
-		}
-
-		ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
-		InputViewDataInputStreamWrapper in = new InputViewDataInputStreamWrapper(
-				new DataInputStream(bais)
-		);
-		initialValue = outSerializer.deserialize(in);
-	}
-
-	@Override
-	public void apply(K k, W window, Iterable<T> values, Collector<R> out) throws Exception {
-		R result = outSerializer.copy(initialValue);
-
-		for (T val: values) {
-			result = wrappedFunction.fold(result, val);
-		}
-
-		out.collect(result);
-	}
-
-	@Override
-	public void setOutputType(TypeInformation<R> outTypeInfo, ExecutionConfig executionConfig) {
-		outSerializer = outTypeInfo.createSerializer(executionConfig);
-
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		OutputViewDataOutputStreamWrapper out = new OutputViewDataOutputStreamWrapper(
-				new DataOutputStream(baos)
-		);
-
-		try {
-			outSerializer.serialize(initialValue, out);
-		} catch (IOException ioe) {
-			throw new RuntimeException("Unable to serialize initial value of type " +
-					initialValue.getClass().getSimpleName() + " of fold window function.", ioe);
-		}
-
-		serializedInitialValue = baos.toByteArray();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.java
deleted file mode 100644
index 24855a5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceAllWindowFunction.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.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 ReduceAllWindowFunction<W extends Window, T> extends RichAllWindowFunction<T, T, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final ReduceFunction<T> reduceFunction;
-
-	public ReduceAllWindowFunction(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 apply(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/06f6ac5d/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
deleted file mode 100644
index edd8a34..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.functions.windowing;
-
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.operators.translation.WrappingFunction;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-public class ReduceWindowFunction<K, W extends Window, T>
-		extends WrappingFunction<ReduceFunction<T>>
-		implements WindowFunction<T, T, K, W> {
-	private static final long serialVersionUID = 1L;
-
-	public ReduceWindowFunction(ReduceFunction<T> reduceFunction) {
-		super(reduceFunction);
-	}
-
-	@Override
-	public void apply(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 = wrappedFunction.reduce(result, v);
-			}
-		}
-
-		if (result != null) {
-			out.collect(result);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java
deleted file mode 100644
index 6a472b1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunctionWithWindow.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.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.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.windowing.windows.Window;
-import org.apache.flink.util.Collector;
-
-public class ReduceWindowFunctionWithWindow<K, W extends Window, T> extends RichWindowFunction<T, Tuple2<W, T>, K, W> {
-	private static final long serialVersionUID = 1L;
-
-	private final ReduceFunction<T> reduceFunction;
-
-	public ReduceWindowFunctionWithWindow(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 apply(K k, W window, Iterable<T> values, Collector<Tuple2<W, 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(Tuple2.of(window, result));
-		}
-	}
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index eda12c0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/WindowFunction.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.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 keyed (grouped) windows.
- *
- * @param <IN> The type of the input value.
- * @param <OUT> The type of the output value.
- * @param <KEY> The type of the key.
- */
-public interface WindowFunction<IN, OUT, KEY, W extends Window> extends Function, Serializable {
-
-	/**
-	 * Evaluates the window and outputs none or several elements.
-	 *
-	 * @param key The key for which this window is evaluated.
-	 * @param window The window that is being evaluated.
-	 * @param values The elements in the window being evaluated.
-	 * @param out A collector for emitting elements.
-	 * 
-	 * @throws Exception The function may throw exceptions to fail the program and trigger recovery. 
-	 */
-	void apply(KEY key, W window, Iterable<IN> values, Collector<OUT> out) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java
deleted file mode 100644
index 86a12e2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/CosineDistance.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.windowing.delta;
-
-import org.apache.flink.streaming.api.functions.windowing.delta.extractor.Extractor;
-
-/**
- * This delta function calculates the cosine distance between two given vectors.
- * The cosine distance is defined as: cosineDistance=1-cosineSimilarity
- * 
- * Cosine similarity: http://en.wikipedia.org/wiki/Cosine_similarity
- * 
- * @param <DATA>
- *            The input data type. This delta function works with a double[],
- *            but can extract/convert to it from any other given object in case
- *            the respective extractor has been set. See
- *            {@link ExtractionAwareDeltaFunction} for more information.
- */
-public class CosineDistance<DATA> extends ExtractionAwareDeltaFunction<DATA, double[]> {
-
-	/**
-	 * auto-generated id
-	 */
-	private static final long serialVersionUID = -1217813582965151599L;
-
-	public CosineDistance() {
-		super(null);
-	}
-
-	public CosineDistance(Extractor<DATA, double[]> converter) {
-		super(converter);
-	}
-
-	@Override
-	public double getNestedDelta(double[] oldDataPoint, double[] newDataPoint) {
-		if (isNullvector(oldDataPoint, newDataPoint)) {
-			return 0;
-		}
-
-		if (oldDataPoint.length != newDataPoint.length) {
-			throw new IllegalArgumentException(
-					"The size of two input arrays are not same, can not compute cosine distance");
-		}
-
-		double sum1 = 0;
-		double sum2 = 0;
-		for (int i = 0; i < oldDataPoint.length; i++) {
-			sum1 += oldDataPoint[i] * oldDataPoint[i];
-			sum2 += newDataPoint[i] * newDataPoint[i];
-		}
-		sum1 = Math.sqrt(sum1);
-		sum2 = Math.sqrt(sum2);
-
-		return 1d - (dotProduct(oldDataPoint, newDataPoint) / (sum1 * sum2));
-	}
-
-	private double dotProduct(double[] a, double[] b) {
-		double result = 0;
-		for (int i = 0; i < a.length; i++) {
-			result += a[i] * b[i];
-		}
-		return result;
-	}
-
-	private boolean isNullvector(double[]... vectors) {
-		outer: for (double[] v : vectors) {
-			for (double field : v) {
-				if (field != 0) {
-					continue outer;
-				}
-			}
-			// This position is only reached in case all fields are 0.
-			return true;
-		}
-		return false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/DeltaFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/DeltaFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/DeltaFunction.java
deleted file mode 100644
index 0ce2bf9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/DeltaFunction.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.windowing.delta;
-
-import java.io.Serializable;
-
-/**
- * This interface allows the implementation of a function which calculates the
- * delta between two data points. Delta functions might be used in delta
- * policies and allow flexible adaptive windowing based on the arriving data
- * points.
- *
- * @param <DATA>
- *            The type of input data which can be compared using this function.
- */
-public interface DeltaFunction<DATA> extends Serializable {
-
-	/**
-	 * Calculates the delta between two given data points.
-	 * 
-	 * @param oldDataPoint
-	 *            the old data point.
-	 * @param newDataPoint
-	 *            the new data point.
-	 * @return the delta between the two given points.
-	 */
-	public double getDelta(DATA oldDataPoint, DATA newDataPoint);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.java
deleted file mode 100644
index 23efbf2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/EuclideanDistance.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.functions.windowing.delta;
-
-import org.apache.flink.streaming.api.functions.windowing.delta.extractor.Extractor;
-
-/**
- * This delta function calculates the euclidean distance between two given
- * points.
- * 
- * Euclidean distance: http://en.wikipedia.org/wiki/Euclidean_distance
- * 
- * @param <DATA>
- *            The input data type. This delta function works with a double[],
- *            but can extract/convert to it from any other given object in case
- *            the respective extractor has been set. See
- *            {@link ExtractionAwareDeltaFunction} for more information.
- */
-public class EuclideanDistance<DATA> extends ExtractionAwareDeltaFunction<DATA, double[]> {
-
-	public EuclideanDistance() {
-		super(null);
-	}
-
-	public EuclideanDistance(Extractor<DATA, double[]> converter) {
-		super(converter);
-	}
-
-	/**
-	 * auto-generated version id
-	 */
-	private static final long serialVersionUID = 3119432599634512359L;
-
-	@Override
-	public double getNestedDelta(double[] oldDataPoint, double[] newDataPoint) {
-		double result = 0;
-		for (int i = 0; i < oldDataPoint.length; i++) {
-			result += (oldDataPoint[i] - newDataPoint[i]) * (oldDataPoint[i] - newDataPoint[i]);
-		}
-		return Math.sqrt(result);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java
deleted file mode 100644
index 7a4e01a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/ExtractionAwareDeltaFunction.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.windowing.delta;
-
-import org.apache.flink.streaming.api.functions.windowing.delta.extractor.Extractor;
-
-/**
- * Extend this abstract class to implement a delta function which is aware of
- * extracting the data on which the delta is calculated from a more complex data
- * structure. For example in case you want to be able to run a delta only on one
- * field of a Tuple type or only on some fields from an array.
- * 
- * @param <DATA>
- *            The input data type. The input of this type will be passed to the
- *            extractor which will transform into a TO-object. The delta
- *            function then runs on this TO-object.
- * @param <TO>
- *            The type on which the delta function runs. (The type of the delta
- *            function)
- */
-public abstract class ExtractionAwareDeltaFunction<DATA, TO> implements DeltaFunction<DATA> {
-
-	/**
-	 * Generated Version ID
-	 */
-	private static final long serialVersionUID = 6927486219702689554L;
-	private Extractor<DATA, TO> converter;
-
-	public ExtractionAwareDeltaFunction(Extractor<DATA, TO> converter) {
-		this.converter = converter;
-	}
-
-	/**
-	 * This method takes the two data point and runs the set extractor on it.
-	 * The delta function implemented at {@link #getNestedDelta} is then called
-	 * with the extracted data. In case no extractor is set the input data gets
-	 * passes to {@link #getNestedDelta} as-is. The return value is just
-	 * forwarded from {@link #getNestedDelta}.
-	 * 
-	 * @param oldDataPoint
-	 *            the older data point as raw data (before extraction).
-	 * @param newDataPoint
-	 *            the new data point as raw data (before extraction).
-	 * @return the delta between the two points.
-	 */
-	@SuppressWarnings("unchecked")
-	@Override
-	public double getDelta(DATA oldDataPoint, DATA newDataPoint) {
-		if (converter == null) {
-			// In case no conversion/extraction is required, we can cast DATA to
-			// TO
-			// => Therefore, "unchecked" warning is suppressed for this method.
-			return getNestedDelta((TO) oldDataPoint, (TO) newDataPoint);
-		} else {
-			return getNestedDelta(converter.extract(oldDataPoint), converter.extract(newDataPoint));
-		}
-
-	}
-
-	/**
-	 * This method is exactly the same as
-	 * {@link DeltaFunction#getDelta(Object, Object)} except that it gets the
-	 * result of the previously done extractions as input. Therefore, this
-	 * method only does the actual calculation of the delta but no data
-	 * extraction or conversion.
-	 * 
-	 * @param oldDataPoint
-	 *            the older data point.
-	 * @param newDataPoint
-	 *            the new data point.
-	 * @return the delta between the two points.
-	 */
-	public abstract double getNestedDelta(TO oldDataPoint, TO newDataPoint);
-
-}

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.java
deleted file mode 100644
index fc7f3ab..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/delta/extractor/FieldsFromTuple.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.windowing.delta.extractor;
-
-import org.apache.flink.api.java.tuple.Tuple;
-
-/**
- * Extracts one or more fields of the type Double from a tuple and puts them
- * into a new double[]
- */
-public class FieldsFromTuple implements Extractor<Tuple, double[]> {
-
-	/**
-	 * auto generated version id
-	 */
-	private static final long serialVersionUID = -2554079091050273761L;
-	int[] indexes;
-
-	/**
-	 * Extracts one or more fields of the the type Double from a tuple and puts
-	 * them into a new double[] (in the specified order).
-	 * 
-	 * @param indexes
-	 *            The indexes of the fields to be extracted.
-	 */
-	public FieldsFromTuple(int... indexes) {
-		this.indexes = indexes;
-	}
-
-	@Override
-	public double[] extract(Tuple in) {
-		double[] out = new double[indexes.length];
-		for (int i = 0; i < indexes.length; i++) {
-			out[i] = (Double) in.getField(indexes[i]);
-		}
-		return out;
-	}
-}


[37/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java
deleted file mode 100644
index 743ee4a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.sling.commons.json.JSONArray;
-import org.apache.sling.commons.json.JSONException;
-import org.apache.sling.commons.json.JSONObject;
-
-public class JSONGenerator {
-
-	public static final String STEPS = "step_function";
-	public static final String ID = "id";
-	public static final String SIDE = "side";
-	public static final String SHIP_STRATEGY = "ship_strategy";
-	public static final String PREDECESSORS = "predecessors";
-	public static final String TYPE = "type";
-	public static final String PACT = "pact";
-	public static final String CONTENTS = "contents";
-	public static final String PARALLELISM = "parallelism";
-
-	private StreamGraph streamGraph;
-
-	public JSONGenerator(StreamGraph streamGraph) {
-		this.streamGraph = streamGraph;
-	}
-
-	public String getJSON() throws JSONException {
-		JSONObject json = new JSONObject();
-		JSONArray nodes = new JSONArray();
-		json.put("nodes", nodes);
-		List<Integer> operatorIDs = new ArrayList<Integer>(streamGraph.getVertexIDs());
-		Collections.sort(operatorIDs, new Comparator<Integer>() {
-			@Override
-			public int compare(Integer o1, Integer o2) {
-				// put sinks at the back
-				if (streamGraph.getSinkIDs().contains(o1)) {
-					return 1;
-				} else if (streamGraph.getSinkIDs().contains(o2)) {
-					return -1;
-				} else {
-					return o1 - o2;
-				}
-			}
-		});
-		visit(nodes, operatorIDs, new HashMap<Integer, Integer>());
-		return json.toString();
-	}
-
-	private void visit(JSONArray jsonArray, List<Integer> toVisit,
-			Map<Integer, Integer> edgeRemapings) throws JSONException {
-
-		Integer vertexID = toVisit.get(0);
-		StreamNode vertex = streamGraph.getStreamNode(vertexID);
-
-		if (streamGraph.getSourceIDs().contains(vertexID)
-				|| Collections.disjoint(vertex.getInEdges(), toVisit)) {
-
-			JSONObject node = new JSONObject();
-			decorateNode(vertexID, node);
-
-			if (!streamGraph.getSourceIDs().contains(vertexID)) {
-				JSONArray inputs = new JSONArray();
-				node.put(PREDECESSORS, inputs);
-
-				for (StreamEdge inEdge : vertex.getInEdges()) {
-					int inputID = inEdge.getSourceId();
-
-					Integer mappedID = (edgeRemapings.keySet().contains(inputID)) ? edgeRemapings
-							.get(inputID) : inputID;
-					decorateEdge(inputs, vertexID, mappedID, inputID);
-				}
-			}
-			jsonArray.put(node);
-			toVisit.remove(vertexID);
-		} else {
-			Integer iterationHead = -1;
-			for (StreamEdge inEdge : vertex.getInEdges()) {
-				int operator = inEdge.getSourceId();
-
-				if (streamGraph.vertexIDtoLoopTimeout.containsKey(operator)) {
-					iterationHead = operator;
-				}
-			}
-
-			JSONObject obj = new JSONObject();
-			JSONArray iterationSteps = new JSONArray();
-			obj.put(STEPS, iterationSteps);
-			obj.put(ID, iterationHead);
-			obj.put(PACT, "IterativeDataStream");
-			obj.put(PARALLELISM, streamGraph.getStreamNode(iterationHead).getParallelism());
-			obj.put(CONTENTS, "Stream Iteration");
-			JSONArray iterationInputs = new JSONArray();
-			obj.put(PREDECESSORS, iterationInputs);
-			toVisit.remove(iterationHead);
-			visitIteration(iterationSteps, toVisit, iterationHead, edgeRemapings, iterationInputs);
-			jsonArray.put(obj);
-		}
-
-		if (!toVisit.isEmpty()) {
-			visit(jsonArray, toVisit, edgeRemapings);
-		}
-	}
-
-	private void visitIteration(JSONArray jsonArray, List<Integer> toVisit, int headId,
-			Map<Integer, Integer> edgeRemapings, JSONArray iterationInEdges) throws JSONException {
-
-		Integer vertexID = toVisit.get(0);
-		StreamNode vertex = streamGraph.getStreamNode(vertexID);
-		toVisit.remove(vertexID);
-
-		// Ignoring head and tail to avoid redundancy
-		if (!streamGraph.vertexIDtoLoopTimeout.containsKey(vertexID)) {
-			JSONObject obj = new JSONObject();
-			jsonArray.put(obj);
-			decorateNode(vertexID, obj);
-			JSONArray inEdges = new JSONArray();
-			obj.put(PREDECESSORS, inEdges);
-
-			for (StreamEdge inEdge : vertex.getInEdges()) {
-				int inputID = inEdge.getSourceId();
-
-				if (edgeRemapings.keySet().contains(inputID)) {
-					decorateEdge(inEdges, vertexID, inputID, inputID);
-				} else if (!streamGraph.vertexIDtoLoopTimeout.containsKey(inputID)) {
-					decorateEdge(iterationInEdges, vertexID, inputID, inputID);
-				}
-			}
-
-			edgeRemapings.put(vertexID, headId);
-			visitIteration(jsonArray, toVisit, headId, edgeRemapings, iterationInEdges);
-		}
-
-	}
-
-	private void decorateEdge(JSONArray inputArray, int vertexID, int mappedInputID, int inputID)
-			throws JSONException {
-		JSONObject input = new JSONObject();
-		inputArray.put(input);
-		input.put(ID, mappedInputID);
-		input.put(SHIP_STRATEGY, streamGraph.getStreamEdge(inputID, vertexID).getPartitioner());
-		input.put(SIDE, (inputArray.length() == 0) ? "first" : "second");
-	}
-
-	private void decorateNode(Integer vertexID, JSONObject node) throws JSONException {
-
-		StreamNode vertex = streamGraph.getStreamNode(vertexID);
-
-		node.put(ID, vertexID);
-		node.put(TYPE, vertex.getOperatorName());
-
-		if (streamGraph.getSourceIDs().contains(vertexID)) {
-			node.put(PACT, "Data Source");
-		} else if (streamGraph.getSinkIDs().contains(vertexID)) {
-			node.put(PACT, "Data Sink");
-		} else {
-			node.put(PACT, "Operator");
-		}
-
-		StreamOperator<?> operator = streamGraph.getStreamNode(vertexID).getOperator();
-
-		node.put(CONTENTS, vertex.getOperatorName());
-
-		node.put(PARALLELISM, streamGraph.getStreamNode(vertexID).getParallelism());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
deleted file mode 100644
index 11bf84f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
+++ /dev/null
@@ -1,468 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.util.ClassLoaderUtil;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskException;
-import org.apache.flink.util.InstantiationUtil;
-
-public class StreamConfig implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	// ------------------------------------------------------------------------
-	//  Config Keys
-	// ------------------------------------------------------------------------
-	
-	private static final String NUMBER_OF_OUTPUTS = "numberOfOutputs";
-	private static final String NUMBER_OF_INPUTS = "numberOfInputs";
-	private static final String CHAINED_OUTPUTS = "chainedOutputs";
-	private static final String CHAINED_TASK_CONFIG = "chainedTaskConfig_";
-	private static final String IS_CHAINED_VERTEX = "isChainedSubtask";
-	private static final String VERTEX_NAME = "vertexID";
-	private static final String ITERATION_ID = "iterationId";
-	private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper";
-	private static final String SERIALIZEDUDF = "serializedUDF";
-	private static final String USER_FUNCTION = "userFunction";
-	private static final String BUFFER_TIMEOUT = "bufferTimeout";
-	private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1";
-	private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2";
-	private static final String TYPE_SERIALIZER_OUT_1 = "typeSerializer_out";
-	private static final String ITERATON_WAIT = "iterationWait";
-	private static final String NONCHAINED_OUTPUTS = "nonChainedOutputs";
-	private static final String EDGES_IN_ORDER = "edgesInOrder";
-	private static final String OUT_STREAM_EDGES = "outStreamEdges";
-	private static final String IN_STREAM_EDGES = "inStreamEdges";
-
-	private static final String CHECKPOINTING_ENABLED = "checkpointing";
-	private static final String CHECKPOINT_MODE = "checkpointMode";
-	
-	private static final String STATE_BACKEND = "statebackend";
-	private static final String STATE_PARTITIONER = "statePartitioner";
-	private static final String STATE_KEY_SERIALIZER = "statekeyser";
-	
-	
-	// ------------------------------------------------------------------------
-	//  Default Values
-	// ------------------------------------------------------------------------
-	
-	private static final long DEFAULT_TIMEOUT = 100;
-	private static final CheckpointingMode DEFAULT_CHECKPOINTING_MODE = CheckpointingMode.EXACTLY_ONCE;
-	
-	
-	// ------------------------------------------------------------------------
-	//  Config
-	// ------------------------------------------------------------------------
-
-	private final Configuration config;
-
-	public StreamConfig(Configuration config) {
-		this.config = config;
-	}
-
-	public Configuration getConfiguration() {
-		return config;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Configured Properties
-	// ------------------------------------------------------------------------
-
-	public void setVertexID(Integer vertexID) {
-		config.setInteger(VERTEX_NAME, vertexID);
-	}
-
-	public Integer getVertexID() {
-		return config.getInteger(VERTEX_NAME, -1);
-	}
-	
-	public void setTypeSerializerIn1(TypeSerializer<?> serializer) {
-		setTypeSerializer(TYPE_SERIALIZER_IN_1, serializer);
-	}
-
-	public void setTypeSerializerIn2(TypeSerializer<?> serializer) {
-		setTypeSerializer(TYPE_SERIALIZER_IN_2, serializer);
-	}
-
-	public void setTypeSerializerOut(TypeSerializer<?> serializer) {
-		setTypeSerializer(TYPE_SERIALIZER_OUT_1, serializer);
-	}
-	
-	public <T> TypeSerializer<T> getTypeSerializerIn1(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_IN_1, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate serializer.", e);
-		}
-	}
-	
-	public <T> TypeSerializer<T> getTypeSerializerIn2(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_IN_2, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate serializer.", e);
-		}
-	}
-	
-	public <T> TypeSerializer<T> getTypeSerializerOut(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_OUT_1, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate serializer.", e);
-		}
-	}
-
-	private void setTypeSerializer(String key, TypeSerializer<?> typeWrapper) {
-		try {
-			InstantiationUtil.writeObjectToConfig(typeWrapper, this.config, key);
-		} catch (IOException e) {
-			throw new StreamTaskException("Could not serialize type serializer.", e);
-		}
-	}
-
-	public void setBufferTimeout(long timeout) {
-		config.setLong(BUFFER_TIMEOUT, timeout);
-	}
-
-	public long getBufferTimeout() {
-		return config.getLong(BUFFER_TIMEOUT, DEFAULT_TIMEOUT);
-	}
-
-	public void setStreamOperator(StreamOperator<?> operator) {
-		if (operator != null) {
-			config.setClass(USER_FUNCTION, operator.getClass());
-
-			try {
-				InstantiationUtil.writeObjectToConfig(operator, this.config, SERIALIZEDUDF);
-			} catch (IOException e) {
-				throw new StreamTaskException("Cannot serialize operator object "
-						+ operator.getClass() + ".", e);
-			}
-		}
-	}
-	
-	public <T> T getStreamOperator(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, SERIALIZEDUDF, cl);
-		}
-		catch (ClassNotFoundException e) {
-			String classLoaderInfo = ClassLoaderUtil.getUserCodeClassLoaderInfo(cl);
-			boolean loadableDoubleCheck = ClassLoaderUtil.validateClassLoadable(e, cl);
-			
-			String exceptionMessage = "Cannot load user class: " + e.getMessage()
-					+ "\nClassLoader info: " + classLoaderInfo + 
-					(loadableDoubleCheck ? 
-							"\nClass was actually found in classloader - deserialization issue." :
-							"\nClass not resolvable through given classloader.");
-			
-			throw new StreamTaskException(exceptionMessage);
-		}
-		catch (Exception e) {
-			throw new StreamTaskException("Cannot instantiate user function.", e);
-		}
-	}
-
-	public void setOutputSelectorWrapper(OutputSelectorWrapper<?> outputSelectorWrapper) {
-		try {
-			InstantiationUtil.writeObjectToConfig(outputSelectorWrapper, this.config, OUTPUT_SELECTOR_WRAPPER);
-		} catch (IOException e) {
-			throw new StreamTaskException("Cannot serialize OutputSelectorWrapper.", e);
-		}
-	}
-	
-	public <T> OutputSelectorWrapper<T> getOutputSelectorWrapper(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, OUTPUT_SELECTOR_WRAPPER, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Cannot deserialize and instantiate OutputSelectorWrapper.", e);
-		}
-	}
-
-	public void setIterationId(String iterationId) {
-		config.setString(ITERATION_ID, iterationId);
-	}
-
-	public String getIterationId() {
-		return config.getString(ITERATION_ID, "");
-	}
-
-	public void setIterationWaitTime(long time) {
-		config.setLong(ITERATON_WAIT, time);
-	}
-
-	public long getIterationWaitTime() {
-		return config.getLong(ITERATON_WAIT, 0);
-	}
-
-	public void setNumberOfInputs(int numberOfInputs) {
-		config.setInteger(NUMBER_OF_INPUTS, numberOfInputs);
-	}
-
-	public int getNumberOfInputs() {
-		return config.getInteger(NUMBER_OF_INPUTS, 0);
-	}
-
-	public void setNumberOfOutputs(int numberOfOutputs) {
-		config.setInteger(NUMBER_OF_OUTPUTS, numberOfOutputs);
-	}
-
-	public int getNumberOfOutputs() {
-		return config.getInteger(NUMBER_OF_OUTPUTS, 0);
-	}
-
-	public void setNonChainedOutputs(List<StreamEdge> outputvertexIDs) {
-		try {
-			InstantiationUtil.writeObjectToConfig(outputvertexIDs, this.config, NONCHAINED_OUTPUTS);
-		} catch (IOException e) {
-			throw new StreamTaskException("Cannot serialize non chained outputs.", e);
-		}
-	}
-	
-	public List<StreamEdge> getNonChainedOutputs(ClassLoader cl) {
-		try {
-			List<StreamEdge> nonChainedOutputs = InstantiationUtil.readObjectFromConfig(this.config, NONCHAINED_OUTPUTS, cl);
-			return nonChainedOutputs == null ?  new ArrayList<StreamEdge>() : nonChainedOutputs;
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate non chained outputs.", e);
-		}
-	}
-
-	public void setChainedOutputs(List<StreamEdge> chainedOutputs) {
-		try {
-			InstantiationUtil.writeObjectToConfig(chainedOutputs, this.config, CHAINED_OUTPUTS);
-		} catch (IOException e) {
-			throw new StreamTaskException("Cannot serialize chained outputs.", e);
-		}
-	}
-	
-	public List<StreamEdge> getChainedOutputs(ClassLoader cl) {
-		try {
-			List<StreamEdge> chainedOutputs = InstantiationUtil.readObjectFromConfig(this.config, CHAINED_OUTPUTS, cl);
-			return chainedOutputs == null ? new ArrayList<StreamEdge>() : chainedOutputs;
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate chained outputs.", e);
-		}
-	}
-
-	public void setOutEdges(List<StreamEdge> outEdges) {
-		try {
-			InstantiationUtil.writeObjectToConfig(outEdges, this.config, OUT_STREAM_EDGES);
-		} catch (IOException e) {
-			throw new StreamTaskException("Cannot serialize outward edges.", e);
-		}
-	}
-	
-	public List<StreamEdge> getOutEdges(ClassLoader cl) {
-		try {
-			List<StreamEdge> outEdges = InstantiationUtil.readObjectFromConfig(this.config, OUT_STREAM_EDGES, cl);
-			return outEdges == null ? new ArrayList<StreamEdge>() : outEdges;
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate outputs.", e);
-		}
-	}
-
-	public void setInPhysicalEdges(List<StreamEdge> inEdges) {
-		try {
-			InstantiationUtil.writeObjectToConfig(inEdges, this.config, IN_STREAM_EDGES);
-		} catch (IOException e) {
-			throw new StreamTaskException("Cannot serialize inward edges.", e);
-		}
-	}
-	
-	public List<StreamEdge> getInPhysicalEdges(ClassLoader cl) {
-		try {
-			List<StreamEdge> inEdges = InstantiationUtil.readObjectFromConfig(this.config, IN_STREAM_EDGES, cl);
-			return inEdges == null ? new ArrayList<StreamEdge>() : inEdges;
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate inputs.", e);
-		}
-	}
-
-	// --------------------- checkpointing -----------------------
-	
-	public void setCheckpointingEnabled(boolean enabled) {
-		config.setBoolean(CHECKPOINTING_ENABLED, enabled);
-	}
-
-	public boolean isCheckpointingEnabled() {
-		return config.getBoolean(CHECKPOINTING_ENABLED, false);
-	}
-	
-	public void setCheckpointMode(CheckpointingMode mode) {
-		config.setInteger(CHECKPOINT_MODE, mode.ordinal());
-	}
-
-	public CheckpointingMode getCheckpointMode() {
-		int ordinal = config.getInteger(CHECKPOINT_MODE, -1);
-		if (ordinal >= 0) {
-			return CheckpointingMode.values()[ordinal];
-		} else {
-			return DEFAULT_CHECKPOINTING_MODE; 
-		}
-	}
-	
-
-	public void setOutEdgesInOrder(List<StreamEdge> outEdgeList) {
-		try {
-			InstantiationUtil.writeObjectToConfig(outEdgeList, this.config, EDGES_IN_ORDER);
-		} catch (IOException e) {
-			throw new StreamTaskException("Could not serialize outputs in order.", e);
-		}
-	}
-	
-	public List<StreamEdge> getOutEdgesInOrder(ClassLoader cl) {
-		try {
-			List<StreamEdge> outEdgesInOrder = InstantiationUtil.readObjectFromConfig(this.config, EDGES_IN_ORDER, cl);
-			return outEdgesInOrder == null ? new ArrayList<StreamEdge>() : outEdgesInOrder;
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate outputs in order.", e);
-		}
-	}
-
-	public void setTransitiveChainedTaskConfigs(Map<Integer, StreamConfig> chainedTaskConfigs) {
-
-		try {
-			InstantiationUtil.writeObjectToConfig(chainedTaskConfigs, this.config, CHAINED_TASK_CONFIG);
-		} catch (IOException e) {
-			throw new StreamTaskException("Could not serialize configuration.", e);
-		}
-	}
-	
-	public Map<Integer, StreamConfig> getTransitiveChainedTaskConfigs(ClassLoader cl) {
-		try {
-			Map<Integer, StreamConfig> confs = InstantiationUtil.readObjectFromConfig(this.config, CHAINED_TASK_CONFIG, cl);
-			return confs == null ? new HashMap<Integer, StreamConfig>() : confs;
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate configuration.", e);
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  State backend
-	// ------------------------------------------------------------------------
-	
-	public void setStateBackend(StateBackend<?> backend) {
-		try {
-			InstantiationUtil.writeObjectToConfig(backend, this.config, STATE_BACKEND);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not serialize stateHandle provider.", e);
-		}
-	}
-	
-	public StateBackend<?> getStateBackend(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, STATE_BACKEND, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate statehandle provider.", e);
-		}
-	}
-	
-	public void setStatePartitioner(KeySelector<?, ?> partitioner) {
-		try {
-			InstantiationUtil.writeObjectToConfig(partitioner, this.config, STATE_PARTITIONER);
-		} catch (IOException e) {
-			throw new StreamTaskException("Could not serialize state partitioner.", e);
-		}
-	}
-	
-	public KeySelector<?, Serializable> getStatePartitioner(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, STATE_PARTITIONER, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate state partitioner.", e);
-		}
-	}
-	
-	public void setStateKeySerializer(TypeSerializer<?> serializer) {
-		try {
-			InstantiationUtil.writeObjectToConfig(serializer, this.config, STATE_KEY_SERIALIZER);
-		} catch (IOException e) {
-			throw new StreamTaskException("Could not serialize state key serializer.", e);
-		}
-	}
-
-	public <K> TypeSerializer<K> getStateKeySerializer(ClassLoader cl) {
-		try {
-			return InstantiationUtil.readObjectFromConfig(this.config, STATE_KEY_SERIALIZER, cl);
-		} catch (Exception e) {
-			throw new StreamTaskException("Could not instantiate state key serializer from task config.", e);
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Miscellansous
-	// ------------------------------------------------------------------------
-	
-	public void setChainStart() {
-		config.setBoolean(IS_CHAINED_VERTEX, true);
-	}
-
-	public boolean isChainStart() {
-		return config.getBoolean(IS_CHAINED_VERTEX, false);
-	}
-
-	@Override
-	public String toString() {
-
-		ClassLoader cl = getClass().getClassLoader();
-
-		StringBuilder builder = new StringBuilder();
-		builder.append("\n=======================");
-		builder.append("Stream Config");
-		builder.append("=======================");
-		builder.append("\nNumber of non-chained inputs: ").append(getNumberOfInputs());
-		builder.append("\nNumber of non-chained outputs: ").append(getNumberOfOutputs());
-		builder.append("\nOutput names: ").append(getNonChainedOutputs(cl));
-		builder.append("\nPartitioning:");
-		for (StreamEdge output : getNonChainedOutputs(cl)) {
-			int outputname = output.getTargetId();
-			builder.append("\n\t").append(outputname).append(": ").append(output.getPartitioner());
-		}
-
-		builder.append("\nChained subtasks: ").append(getChainedOutputs(cl));
-
-		try {
-			builder.append("\nOperator: ").append(getStreamOperator(cl).getClass().getSimpleName());
-		}
-		catch (Exception e) {
-			builder.append("\nOperator: Missing");
-		}
-		builder.append("\nBuffer timeout: ").append(getBufferTimeout());
-		builder.append("\nState Monitoring: ").append(isCheckpointingEnabled());
-		if (isChainStart() && getChainedOutputs(cl).size() > 0) {
-			builder.append("\n\n\n---------------------\nChained task configs\n---------------------\n");
-			builder.append(getTransitiveChainedTaskConfigs(cl));
-		}
-
-		return builder.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java
deleted file mode 100644
index c252870..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamEdge.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-
-/**
- * An edge in the streaming topology. One edge like this does not necessarily
- * gets converted to a connection between two job vertices (due to
- * chaining/optimization).
- */
-public class StreamEdge implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	final private String edgeId;
-
-	final private StreamNode sourceVertex;
-	final private StreamNode targetVertex;
-
-	/**
-	 * The type number of the input for co-tasks.
-	 */
-	final private int typeNumber;
-
-	/**
-	 * A list of output names that the target vertex listens to (if there is
-	 * output selection).
-	 */
-	private final List<String> selectedNames;
-	private StreamPartitioner<?> outputPartitioner;
-
-	public StreamEdge(StreamNode sourceVertex, StreamNode targetVertex, int typeNumber,
-			List<String> selectedNames, StreamPartitioner<?> outputPartitioner) {
-		this.sourceVertex = sourceVertex;
-		this.targetVertex = targetVertex;
-		this.typeNumber = typeNumber;
-		this.selectedNames = selectedNames;
-		this.outputPartitioner = outputPartitioner;
-
-		this.edgeId = sourceVertex + "_" + targetVertex + "_" + typeNumber + "_" + selectedNames
-				+ "_" + outputPartitioner;
-	}
-
-	public StreamNode getSourceVertex() {
-		return sourceVertex;
-	}
-
-	public StreamNode getTargetVertex() {
-		return targetVertex;
-	}
-
-	public int getSourceId() {
-		return sourceVertex.getId();
-	}
-
-	public int getTargetId() {
-		return targetVertex.getId();
-	}
-
-	public int getTypeNumber() {
-		return typeNumber;
-	}
-
-	public List<String> getSelectedNames() {
-		return selectedNames;
-	}
-
-	public StreamPartitioner<?> getPartitioner() {
-		return outputPartitioner;
-	}
-	
-	public void setPartitioner(StreamPartitioner<?> partitioner) {
-		this.outputPartitioner = partitioner;
-	}
-
-	@Override
-	public int hashCode() {
-		return edgeId.hashCode();
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		StreamEdge that = (StreamEdge) o;
-
-		return edgeId.equals(that.edgeId);
-	}
-
-	@Override
-	public String toString() {
-		return "(" + sourceVertex + " -> " + targetVertex + ", typeNumber=" + typeNumber
-				+ ", selectedNames=" + selectedNames + ", outputPartitioner=" + outputPartitioner
-				+ ')';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
deleted file mode 100644
index be020d7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ /dev/null
@@ -1,619 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.collect.Lists;
-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.io.InputFormat;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.api.java.typeutils.MissingTypeInfo;
-import org.apache.flink.optimizer.plan.StreamingPlan;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
-import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
-import org.apache.flink.streaming.runtime.tasks.SourceStreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamIterationHead;
-import org.apache.flink.streaming.runtime.tasks.StreamIterationTail;
-import org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask;
-import org.apache.sling.commons.json.JSONException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Class representing the streaming topology. It contains all the information
- * necessary to build the jobgraph for the execution.
- * 
- */
-public class StreamGraph extends StreamingPlan {
-
-	/** The default interval for checkpoints, in milliseconds */
-	public static final int DEFAULT_CHECKPOINTING_INTERVAL_MS = 5000;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(StreamGraph.class);
-
-	private String jobName = StreamExecutionEnvironment.DEFAULT_JOB_NAME;
-
-	private final StreamExecutionEnvironment environemnt;
-	private final ExecutionConfig executionConfig;
-
-	private CheckpointingMode checkpointingMode;
-	private boolean checkpointingEnabled = false;
-	private long checkpointingInterval = DEFAULT_CHECKPOINTING_INTERVAL_MS;
-	private boolean chaining = true;
-
-	private Map<Integer, StreamNode> streamNodes;
-	private Set<Integer> sources;
-	private Set<Integer> sinks;
-	private Map<Integer, Tuple2<Integer, List<String>>> virtualSelectNodes;
-	private Map<Integer, Tuple2<Integer, StreamPartitioner<?>>> virtuaPartitionNodes;
-
-	protected Map<Integer, String> vertexIDtoBrokerID;
-	protected Map<Integer, Long> vertexIDtoLoopTimeout;
-	private StateBackend<?> stateBackend;
-	private Set<Tuple2<StreamNode, StreamNode>> iterationSourceSinkPairs;
-
-	private boolean forceCheckpoint = false;
-
-	public StreamGraph(StreamExecutionEnvironment environment) {
-
-		this.environemnt = environment;
-		executionConfig = environment.getConfig();
-
-		// create an empty new stream graph.
-		clear();
-	}
-
-	/**
-	 * Remove all registered nodes etc.
-	 */
-	public void clear() {
-		streamNodes = Maps.newHashMap();
-		virtualSelectNodes = Maps.newHashMap();
-		virtuaPartitionNodes = Maps.newHashMap();
-		vertexIDtoBrokerID = Maps.newHashMap();
-		vertexIDtoLoopTimeout = Maps.newHashMap();
-		iterationSourceSinkPairs = Sets.newHashSet();
-		sources = Sets.newHashSet();
-		sinks = Sets.newHashSet();
-	}
-
-	protected ExecutionConfig getExecutionConfig() {
-		return executionConfig;
-	}
-
-	public void setJobName(String jobName) {
-		this.jobName = jobName;
-	}
-
-	public void setChaining(boolean chaining) {
-		this.chaining = chaining;
-	}
-
-	public void setCheckpointingEnabled(boolean checkpointingEnabled) {
-		this.checkpointingEnabled = checkpointingEnabled;
-	}
-
-	public void setCheckpointingInterval(long checkpointingInterval) {
-		this.checkpointingInterval = checkpointingInterval;
-	}
-
-	public void forceCheckpoint() {
-		this.forceCheckpoint = true;
-	}
-
-	public void setStateBackend(StateBackend<?> backend) {
-		this.stateBackend = backend;
-	}
-
-	public StateBackend<?> getStateBackend() {
-		return this.stateBackend;
-	}
-
-	public long getCheckpointingInterval() {
-		return checkpointingInterval;
-	}
-
-	// Checkpointing
-	
-	public boolean isChainingEnabled() {
-		return chaining;
-	}
-
-	public boolean isCheckpointingEnabled() {
-		return checkpointingEnabled;
-	}
-
-	public CheckpointingMode getCheckpointingMode() {
-		return checkpointingMode;
-	}
-
-	public void setCheckpointingMode(CheckpointingMode checkpointingMode) {
-		this.checkpointingMode = checkpointingMode;
-	}
-	
-
-	public boolean isIterative() {
-		return!vertexIDtoLoopTimeout.isEmpty();
-	}
-
-	public <IN, OUT> void addSource(Integer vertexID, StreamOperator<OUT> operatorObject,
-			TypeInformation<IN> inTypeInfo, TypeInformation<OUT> outTypeInfo, String operatorName) {
-		addOperator(vertexID, operatorObject, inTypeInfo, outTypeInfo, operatorName);
-		sources.add(vertexID);
-	}
-
-	public <IN, OUT> void addSink(Integer vertexID, StreamOperator<OUT> operatorObject,
-			TypeInformation<IN> inTypeInfo, TypeInformation<OUT> outTypeInfo, String operatorName) {
-		addOperator(vertexID, operatorObject, inTypeInfo, outTypeInfo, operatorName);
-		sinks.add(vertexID);
-	}
-
-	public <IN, OUT> void addOperator(
-			Integer vertexID,
-			StreamOperator<OUT> operatorObject,
-			TypeInformation<IN> inTypeInfo,
-			TypeInformation<OUT> outTypeInfo,
-			String operatorName) {
-
-		if (operatorObject instanceof StreamSource) {
-			addNode(vertexID, SourceStreamTask.class, operatorObject, operatorName);
-		} else {
-			addNode(vertexID, OneInputStreamTask.class, operatorObject, operatorName);
-		}
-
-		TypeSerializer<IN> inSerializer = inTypeInfo != null && !(inTypeInfo instanceof MissingTypeInfo) ? inTypeInfo.createSerializer(executionConfig) : null;
-
-		TypeSerializer<OUT> outSerializer = outTypeInfo != null && !(outTypeInfo instanceof MissingTypeInfo) ? outTypeInfo.createSerializer(executionConfig) : null;
-
-		setSerializers(vertexID, inSerializer, null, outSerializer);
-
-		if (operatorObject instanceof OutputTypeConfigurable) {
-			@SuppressWarnings("unchecked")
-			OutputTypeConfigurable<OUT> outputTypeConfigurable = (OutputTypeConfigurable<OUT>) operatorObject;
-			// sets the output type which must be know at StreamGraph creation time
-			outputTypeConfigurable.setOutputType(outTypeInfo, executionConfig);
-		}
-
-		if (operatorObject instanceof InputTypeConfigurable) {
-			InputTypeConfigurable inputTypeConfigurable = (InputTypeConfigurable) operatorObject;
-			inputTypeConfigurable.setInputType(inTypeInfo, executionConfig);
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Vertex: {}", vertexID);
-		}
-	}
-
-	public <IN1, IN2, OUT> void addCoOperator(
-			Integer vertexID,
-			TwoInputStreamOperator<IN1, IN2, OUT> taskOperatorObject,
-			TypeInformation<IN1> in1TypeInfo,
-			TypeInformation<IN2> in2TypeInfo,
-			TypeInformation<OUT> outTypeInfo,
-			String operatorName) {
-
-		addNode(vertexID, TwoInputStreamTask.class, taskOperatorObject, operatorName);
-
-		TypeSerializer<OUT> outSerializer = (outTypeInfo != null) && !(outTypeInfo instanceof MissingTypeInfo) ?
-				outTypeInfo.createSerializer(executionConfig) : null;
-
-		setSerializers(vertexID, in1TypeInfo.createSerializer(executionConfig), in2TypeInfo.createSerializer(executionConfig), outSerializer);
-
-		if (taskOperatorObject instanceof OutputTypeConfigurable) {
-			@SuppressWarnings("unchecked")
-			OutputTypeConfigurable<OUT> outputTypeConfigurable = (OutputTypeConfigurable<OUT>) taskOperatorObject;
-			// sets the output type which must be know at StreamGraph creation time
-			outputTypeConfigurable.setOutputType(outTypeInfo, executionConfig);
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("CO-TASK: {}", vertexID);
-		}
-	}
-
-	protected StreamNode addNode(Integer vertexID, Class<? extends AbstractInvokable> vertexClass,
-			StreamOperator<?> operatorObject, String operatorName) {
-
-		if (streamNodes.containsKey(vertexID)) {
-			throw new RuntimeException("Duplicate vertexID " + vertexID);
-		}
-
-		StreamNode vertex = new StreamNode(environemnt, vertexID, operatorObject, operatorName,
-				new ArrayList<OutputSelector<?>>(), vertexClass);
-
-		streamNodes.put(vertexID, vertex);
-
-		return vertex;
-	}
-
-	/**
-	 * Adds a new virtual node that is used to connect a downstream vertex to only the outputs
-	 * with the selected names.
-	 *
-	 * When adding an edge from the virtual node to a downstream node the connection will be made
-	 * to the original node, only with the selected names given here.
-	 *
-	 * @param originalId ID of the node that should be connected to.
-	 * @param virtualId ID of the virtual node.
-	 * @param selectedNames The selected names.
-	 */
-	public void addVirtualSelectNode(Integer originalId, Integer virtualId, List<String> selectedNames) {
-
-		if (virtualSelectNodes.containsKey(virtualId)) {
-			throw new IllegalStateException("Already has virtual select node with id " + virtualId);
-		}
-
-		virtualSelectNodes.put(virtualId,
-				new Tuple2<Integer, List<String>>(originalId, selectedNames));
-	}
-
-	/**
-	 * Adds a new virtual node that is used to connect a downstream vertex to an input with a certain
-	 * partitioning.
-	 *
-	 * When adding an edge from the virtual node to a downstream node the connection will be made
-	 * to the original node, but with the partitioning given here.
-	 *
-	 * @param originalId ID of the node that should be connected to.
-	 * @param virtualId ID of the virtual node.
-	 * @param partitioner The partitioner
-	 */
-	public void addVirtualPartitionNode(Integer originalId, Integer virtualId, StreamPartitioner<?> partitioner) {
-
-		if (virtuaPartitionNodes.containsKey(virtualId)) {
-			throw new IllegalStateException("Already has virtual partition node with id " + virtualId);
-		}
-
-		virtuaPartitionNodes.put(virtualId,
-				new Tuple2<Integer, StreamPartitioner<?>>(originalId, partitioner));
-	}
-
-	public void addEdge(Integer upStreamVertexID, Integer downStreamVertexID, int typeNumber) {
-		addEdgeInternal(upStreamVertexID,
-				downStreamVertexID,
-				typeNumber,
-				null,
-				Lists.<String>newArrayList());
-
-	}
-
-	private void addEdgeInternal(Integer upStreamVertexID,
-			Integer downStreamVertexID,
-			int typeNumber,
-			StreamPartitioner<?> partitioner,
-			List<String> outputNames) {
-
-
-		if (virtualSelectNodes.containsKey(upStreamVertexID)) {
-			int virtualId = upStreamVertexID;
-			upStreamVertexID = virtualSelectNodes.get(virtualId).f0;
-			if (outputNames.isEmpty()) {
-				// selections that happen downstream override earlier selections
-				outputNames = virtualSelectNodes.get(virtualId).f1;
-			}
-			addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames);
-		} else if (virtuaPartitionNodes.containsKey(upStreamVertexID)) {
-			int virtualId = upStreamVertexID;
-			upStreamVertexID = virtuaPartitionNodes.get(virtualId).f0;
-			if (partitioner == null) {
-				partitioner = virtuaPartitionNodes.get(virtualId).f1;
-			}
-			addEdgeInternal(upStreamVertexID, downStreamVertexID, typeNumber, partitioner, outputNames);
-		} else {
-			StreamNode upstreamNode = getStreamNode(upStreamVertexID);
-			StreamNode downstreamNode = getStreamNode(downStreamVertexID);
-
-			// If no partitioner was specified and the parallelism of upstream and downstream
-			// operator matches use forward partitioning, use rebalance otherwise.
-			if (partitioner == null && upstreamNode.getParallelism() == downstreamNode.getParallelism()) {
-				partitioner = new ForwardPartitioner<Object>();
-			} else if (partitioner == null) {
-				partitioner = new RebalancePartitioner<Object>();
-			}
-
-			if (partitioner instanceof ForwardPartitioner) {
-				if (upstreamNode.getParallelism() != downstreamNode.getParallelism()) {
-					throw new UnsupportedOperationException("Forward partitioning does not allow " +
-							"change of parallelism. Upstream operation: " + upstreamNode + " parallelism: " + upstreamNode.getParallelism() +
-							", downstream operation: " + downstreamNode + " parallelism: " + downstreamNode.getParallelism() +
-							" You must use another partitioning strategy, such as broadcast, rebalance, shuffle or global.");
-				}
-			}
-
-			StreamEdge edge = new StreamEdge(upstreamNode, downstreamNode, typeNumber, outputNames, partitioner);
-
-			getStreamNode(edge.getSourceId()).addOutEdge(edge);
-			getStreamNode(edge.getTargetId()).addInEdge(edge);
-		}
-	}
-
-	public <T> void addOutputSelector(Integer vertexID, OutputSelector<T> outputSelector) {
-		if (virtuaPartitionNodes.containsKey(vertexID)) {
-			addOutputSelector(virtuaPartitionNodes.get(vertexID).f0, outputSelector);
-		} else if (virtualSelectNodes.containsKey(vertexID)) {
-			addOutputSelector(virtualSelectNodes.get(vertexID).f0, outputSelector);
-		} else {
-			getStreamNode(vertexID).addOutputSelector(outputSelector);
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Outputselector set for {}", vertexID);
-			}
-		}
-
-	}
-
-	public void setParallelism(Integer vertexID, int parallelism) {
-		if (getStreamNode(vertexID) != null) {
-			getStreamNode(vertexID).setParallelism(parallelism);
-		}
-	}
-
-	public void setKey(Integer vertexID, KeySelector<?, ?> keySelector, TypeSerializer<?> keySerializer) {
-		StreamNode node = getStreamNode(vertexID);
-		node.setStatePartitioner(keySelector);
-		node.setStateKeySerializer(keySerializer);
-	}
-
-	public void setBufferTimeout(Integer vertexID, long bufferTimeout) {
-		if (getStreamNode(vertexID) != null) {
-			getStreamNode(vertexID).setBufferTimeout(bufferTimeout);
-		}
-	}
-
-	public void setSerializers(Integer vertexID, TypeSerializer<?> in1, TypeSerializer<?> in2, TypeSerializer<?> out) {
-		StreamNode vertex = getStreamNode(vertexID);
-		vertex.setSerializerIn1(in1);
-		vertex.setSerializerIn2(in2);
-		vertex.setSerializerOut(out);
-	}
-
-	public void setSerializersFrom(Integer from, Integer to) {
-		StreamNode fromVertex = getStreamNode(from);
-		StreamNode toVertex = getStreamNode(to);
-
-		toVertex.setSerializerIn1(fromVertex.getTypeSerializerOut());
-		toVertex.setSerializerOut(fromVertex.getTypeSerializerIn1());
-	}
-
-	public <OUT> void setOutType(Integer vertexID, TypeInformation<OUT> outType) {
-		getStreamNode(vertexID).setSerializerOut(outType.createSerializer(executionConfig));
-	}
-
-	public <IN, OUT> void setOperator(Integer vertexID, StreamOperator<OUT> operatorObject) {
-		getStreamNode(vertexID).setOperator(operatorObject);
-	}
-
-	public void setInputFormat(Integer vertexID, InputFormat<?, ?> inputFormat) {
-		getStreamNode(vertexID).setInputFormat(inputFormat);
-	}
-
-	public void setResourceStrategy(Integer vertexID, ResourceStrategy strategy) {
-		StreamNode node = getStreamNode(vertexID);
-		if (node == null) {
-			return;
-		}
-
-		switch (strategy) {
-		case ISOLATE:
-			node.isolateSlot();
-			break;
-		case NEWGROUP:
-			node.startNewSlotSharingGroup();
-			break;
-		default:
-			throw new IllegalArgumentException("Unknown resource strategy");
-		}
-	}
-
-	public StreamNode getStreamNode(Integer vertexID) {
-		return streamNodes.get(vertexID);
-	}
-
-	protected Collection<? extends Integer> getVertexIDs() {
-		return streamNodes.keySet();
-	}
-
-	public StreamEdge getStreamEdge(int sourceId, int targetId) {
-		Iterator<StreamEdge> outIterator = getStreamNode(sourceId).getOutEdges().iterator();
-		while (outIterator.hasNext()) {
-			StreamEdge edge = outIterator.next();
-
-			if (edge.getTargetId() == targetId) {
-				return edge;
-			}
-		}
-
-		throw new RuntimeException("No such edge in stream graph: " + sourceId + " -> " + targetId);
-	}
-
-	public Collection<Integer> getSourceIDs() {
-		return sources;
-	}
-
-
-	public Collection<Integer> getSinkIDs() {
-		return sinks;
-	}
-
-	public Collection<StreamNode> getStreamNodes() {
-		return streamNodes.values();
-	}
-
-	public Set<Tuple2<Integer, StreamOperator<?>>> getOperators() {
-		Set<Tuple2<Integer, StreamOperator<?>>> operatorSet = new HashSet<Tuple2<Integer, StreamOperator<?>>>();
-		for (StreamNode vertex : streamNodes.values()) {
-			operatorSet.add(new Tuple2<Integer, StreamOperator<?>>(vertex.getId(), vertex
-					.getOperator()));
-		}
-		return operatorSet;
-	}
-
-	public String getBrokerID(Integer vertexID) {
-		return vertexIDtoBrokerID.get(vertexID);
-	}
-
-	public long getLoopTimeout(Integer vertexID) {
-		return vertexIDtoLoopTimeout.get(vertexID);
-	}
-
-	public  Tuple2<StreamNode, StreamNode> createIterationSourceAndSink(int loopId, int sourceId, int sinkId, long timeout, int parallelism) {
-
-		StreamNode source = this.addNode(sourceId,
-				StreamIterationHead.class,
-				null,
-				null);
-		sources.add(source.getId());
-		setParallelism(source.getId(), parallelism);
-
-		StreamNode sink = this.addNode(sinkId,
-				StreamIterationTail.class,
-				null,
-				null);
-		sinks.add(sink.getId());
-		setParallelism(sink.getId(), parallelism);
-
-		iterationSourceSinkPairs.add(new Tuple2<StreamNode, StreamNode>(source, sink));
-
-		source.setOperatorName("IterationSource-" + loopId);
-		sink.setOperatorName("IterationSink-" + loopId);
-		this.vertexIDtoBrokerID.put(source.getId(), "broker-" + loopId);
-		this.vertexIDtoBrokerID.put(sink.getId(), "broker-" + loopId);
-		this.vertexIDtoLoopTimeout.put(source.getId(), timeout);
-		this.vertexIDtoLoopTimeout.put(sink.getId(), timeout);
-
-		return new Tuple2<StreamNode, StreamNode>(source, sink);
-	}
-
-	public Set<Tuple2<StreamNode, StreamNode>> getIterationSourceSinkPairs() {
-		return iterationSourceSinkPairs;
-	}
-
-	protected void removeEdge(StreamEdge edge) {
-
-		edge.getSourceVertex().getOutEdges().remove(edge);
-		edge.getTargetVertex().getInEdges().remove(edge);
-
-	}
-
-	protected void removeVertex(StreamNode toRemove) {
-
-		Set<StreamEdge> edgesToRemove = new HashSet<StreamEdge>();
-
-		edgesToRemove.addAll(toRemove.getInEdges());
-		edgesToRemove.addAll(toRemove.getOutEdges());
-
-		for (StreamEdge edge : edgesToRemove) {
-			removeEdge(edge);
-		}
-		streamNodes.remove(toRemove.getId());
-	}
-
-	/**
-	 * Gets the assembled {@link JobGraph} and adds a default name for it.
-	 */
-	public JobGraph getJobGraph() {
-		return getJobGraph(jobName);
-	}
-
-	/**
-	 * Gets the assembled {@link JobGraph} and adds a user specified name for
-	 * it.
-	 * 
-	 * @param jobGraphName
-	 *            name of the jobGraph
-	 */
-	public JobGraph getJobGraph(String jobGraphName) {
-		// temporarily forbid checkpointing for iterative jobs
-		if (isIterative() && isCheckpointingEnabled() && !forceCheckpoint) {
-			throw new UnsupportedOperationException(
-					"Checkpointing is currently not supported by default for iterative jobs, as we cannot guarantee exactly once semantics. "
-							+ "State checkpoints happen normally, but records in-transit during the snapshot will be lost upon failure. "
-							+ "\nThe user can force enable state checkpoints with the reduced guarantees by calling: env.enableCheckpointing(interval,true)");
-		}
-
-		setJobName(jobGraphName);
-
-		StreamingJobGraphGenerator jobgraphGenerator = new StreamingJobGraphGenerator(this);
-
-		return jobgraphGenerator.createJobGraph(jobGraphName);
-	}
-
-	@Override
-	public String getStreamingPlanAsJSON() {
-
-		try {
-			return new JSONGenerator(this).getJSON();
-		} catch (JSONException e) {
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("JSON plan creation failed: {}", e);
-			}
-			return "";
-		}
-
-	}
-
-	@Override
-	public void dumpStreamingPlanAsJSON(File file) throws IOException {
-		PrintWriter pw = null;
-		try {
-			pw = new PrintWriter(new FileOutputStream(file), false);
-			pw.write(getStreamingPlanAsJSON());
-			pw.flush();
-
-		} finally {
-			if (pw != null) {
-				pw.close();
-			}
-		}
-	}
-
-	public static enum ResourceStrategy {
-		DEFAULT, ISOLATE, NEWGROUP
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
deleted file mode 100644
index 4a87eb3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
+++ /dev/null
@@ -1,538 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.api.graph;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.FileSourceFunction;
-import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation;
-import org.apache.flink.streaming.api.transformations.FeedbackTransformation;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.transformations.PartitionTransformation;
-import org.apache.flink.streaming.api.transformations.SelectTransformation;
-import org.apache.flink.streaming.api.transformations.SinkTransformation;
-import org.apache.flink.streaming.api.transformations.SourceTransformation;
-import org.apache.flink.streaming.api.transformations.SplitTransformation;
-import org.apache.flink.streaming.api.transformations.StreamTransformation;
-import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
-import org.apache.flink.streaming.api.transformations.UnionTransformation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A generator that generates a {@link StreamGraph} from a graph of
- * {@link StreamTransformation StreamTransformations}.
- *
- * <p>
- * This traverses the tree of {@code StreamTransformations} starting from the sinks. At each
- * transformation we recursively transform the inputs, then create a node in the {@code StreamGraph}
- * and add edges from the input Nodes to our newly created node. The transformation methods
- * return the IDs of the nodes in the StreamGraph that represent the input transformation. Several
- * IDs can be returned to be able to deal with feedback transformations and unions.
- *
- * <p>
- * Partitioning, split/select and union don't create actual nodes in the {@code StreamGraph}. For
- * these, we create a virtual node in the {@code StreamGraph} that holds the specific property, i.e.
- * partitioning, selector and so on. When an edge is created from a virtual node to a downstream
- * node the {@code StreamGraph} resolved the id of the original node and creates an edge
- * in the graph with the desired property. For example, if you have this graph:
- *
- * <pre>
- *     Map-1 -> HashPartition-2 -> Map-3
- * </pre>
- *
- * where the numbers represent transformation IDs. We first recurse all the way down. {@code Map-1}
- * is transformed, i.e. we create a {@code StreamNode} with ID 1. Then we transform the
- * {@code HashPartition}, for this, we create virtual node of ID 4 that holds the property
- * {@code HashPartition}. This transformation returns the ID 4. Then we transform the {@code Map-3}.
- * We add the edge {@code 4 -> 3}. The {@code StreamGraph} resolved the actual node with ID 1 and
- * creates and edge {@code 1 -> 3} with the property HashPartition.
- */
-public class StreamGraphGenerator {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamGraphGenerator.class);
-
-	// The StreamGraph that is being built, this is initialized at the beginning.
-	private StreamGraph streamGraph;
-
-	private final StreamExecutionEnvironment env;
-
-	// This is used to assign a unique ID to iteration source/sink
-	protected static Integer iterationIdCounter = 0;
-	public static int getNewIterationNodeId() {
-		iterationIdCounter--;
-		return iterationIdCounter;
-	}
-
-	// Keep track of which Transforms we have already transformed, this is necessary because
-	// we have loops, i.e. feedback edges.
-	private Map<StreamTransformation<?>, Collection<Integer>> alreadyTransformed;
-
-
-	/**
-	 * Private constructor. The generator should only be invoked using {@link #generate}.
-	 */
-	private StreamGraphGenerator(StreamExecutionEnvironment env) {
-		this.streamGraph = new StreamGraph(env);
-		this.streamGraph.setChaining(env.isChainingEnabled());
-		
-		if (env.getCheckpointInterval() > 0) {
-			this.streamGraph.setCheckpointingEnabled(true);
-			this.streamGraph.setCheckpointingInterval(env.getCheckpointInterval());
-			this.streamGraph.setCheckpointingMode(env.getCheckpointingMode());
-		}
-		this.streamGraph.setStateBackend(env.getStateBackend());
-		if (env.isForceCheckpointing()) {
-			this.streamGraph.forceCheckpoint();
-		}
-		
-		this.env = env;
-		this.alreadyTransformed = new HashMap<>();
-	}
-
-	/**
-	 * Generates a {@code StreamGraph} by traversing the graph of {@code StreamTransformations}
-	 * starting from the given transformations.
-	 *
-	 * @param env The {@code StreamExecutionEnvironment} that is used to set some parameters of the
-	 *            job
-	 * @param transformations The transformations starting from which to transform the graph
-	 *
-	 * @return The generated {@code StreamGraph}
-	 */
-	public static StreamGraph generate(StreamExecutionEnvironment env, List<StreamTransformation<?>> transformations) {
-		return new StreamGraphGenerator(env).generateInternal(transformations);
-	}
-
-	/**
-	 * This starts the actual transformation, beginning from the sinks.
-	 */
-	private StreamGraph generateInternal(List<StreamTransformation<?>> transformations) {
-		for (StreamTransformation<?> transformation: transformations) {
-			transform(transformation);
-		}
-		return streamGraph;
-	}
-
-	/**
-	 * Transforms one {@code StreamTransformation}.
-	 *
-	 * <p>
-	 * This checks whether we already transformed it and exits early in that case. If not it
-	 * delegates to one of the transformation specific methods.
-	 */
-	private Collection<Integer> transform(StreamTransformation<?> transform) {
-
-		if (alreadyTransformed.containsKey(transform)) {
-			return alreadyTransformed.get(transform);
-		}
-
-		LOG.debug("Transforming " + transform);
-
-		// call at least once to trigger exceptions about MissingTypeInfo
-		transform.getOutputType();
-
-		Collection<Integer> transformedIds;
-		if (transform instanceof OneInputTransformation<?, ?>) {
-			transformedIds = transformOnInputTransform((OneInputTransformation<?, ?>) transform);
-		} else if (transform instanceof TwoInputTransformation<?, ?, ?>) {
-			transformedIds = transformTwoInputTransform((TwoInputTransformation<?, ?, ?>) transform);
-		} else if (transform instanceof SourceTransformation<?>) {
-			transformedIds = transformSource((SourceTransformation<?>) transform);
-		} else if (transform instanceof SinkTransformation<?>) {
-			transformedIds = transformSink((SinkTransformation<?>) transform);
-		} else if (transform instanceof UnionTransformation<?>) {
-			transformedIds = transformUnion((UnionTransformation<?>) transform);
-		} else if (transform instanceof SplitTransformation<?>) {
-			transformedIds = transformSplit((SplitTransformation<?>) transform);
-		} else if (transform instanceof SelectTransformation<?>) {
-			transformedIds = transformSelect((SelectTransformation<?>) transform);
-		} else if (transform instanceof FeedbackTransformation<?>) {
-			transformedIds = transformFeedback((FeedbackTransformation<?>) transform);
-		} else if (transform instanceof CoFeedbackTransformation<?>) {
-			transformedIds = transformCoFeedback((CoFeedbackTransformation<?>) transform);
-		} else if (transform instanceof PartitionTransformation<?>) {
-			transformedIds = transformPartition((PartitionTransformation<?>) transform);
-		} else {
-			throw new IllegalStateException("Unknown transformation: " + transform);
-		}
-
-		// need this check because the iterate transformation adds itself before
-		// transforming the feedback edges
-		if (!alreadyTransformed.containsKey(transform)) {
-			alreadyTransformed.put(transform, transformedIds);
-		}
-
-		if (transform.getBufferTimeout() > 0) {
-			streamGraph.setBufferTimeout(transform.getId(), transform.getBufferTimeout());
-		}
-		if (transform.getResourceStrategy() != StreamGraph.ResourceStrategy.DEFAULT) {
-			streamGraph.setResourceStrategy(transform.getId(), transform.getResourceStrategy());
-		}
-
-		return transformedIds;
-	}
-
-	/**
-	 * Transforms a {@code UnionTransformation}.
-	 *
-	 * <p>
-	 * This is easy, we only have to transform the inputs and return all the IDs in a list so
-	 * that downstream operations can connect to all upstream nodes.
-	 */
-	private <T> Collection<Integer> transformUnion(UnionTransformation<T> union) {
-		List<StreamTransformation<T>> inputs = union.getInputs();
-		List<Integer> resultIds = new ArrayList<>();
-
-		for (StreamTransformation<T> input: inputs) {
-			resultIds.addAll(transform(input));
-		}
-
-		return resultIds;
-	}
-
-	/**
-	 * Transforms a {@code PartitionTransformation}.
-	 *
-	 * <p>
-	 * For this we create a virtual node in the {@code StreamGraph} that holds the partition
-	 * property. @see StreamGraphGenerator
-	 */
-	private <T> Collection<Integer> transformPartition(PartitionTransformation<T> partition) {
-		StreamTransformation<T> input = partition.getInput();
-		List<Integer> resultIds = new ArrayList<>();
-
-		Collection<Integer> transformedIds = transform(input);
-		for (Integer transformedId: transformedIds) {
-			int virtualId = StreamTransformation.getNewNodeId();
-			streamGraph.addVirtualPartitionNode(transformedId, virtualId, partition.getPartitioner());
-			resultIds.add(virtualId);
-		}
-
-		return resultIds;
-	}
-
-	/**
-	 * Transforms a {@code SplitTransformation}.
-	 *
-	 * <p>
-	 * We add the output selector to previously transformed nodes.
-	 */
-	private <T> Collection<Integer> transformSplit(SplitTransformation<T> split) {
-
-		StreamTransformation<T> input = split.getInput();
-		Collection<Integer> resultIds = transform(input);
-
-		// the recursive transform call might have transformed this already
-		if (alreadyTransformed.containsKey(split)) {
-			return alreadyTransformed.get(split);
-		}
-
-		for (int inputId : resultIds) {
-			streamGraph.addOutputSelector(inputId, split.getOutputSelector());
-		}
-
-
-		return resultIds;
-	}
-
-	/**
-	 * Transforms a {@code SelectTransformation}.
-	 *
-	 * <p>
-	 * For this we create a virtual node in the {@code StreamGraph} holds the selected names.
-	 * @see org.apache.flink.streaming.api.graph.StreamGraphGenerator
-	 */
-	private <T> Collection<Integer> transformSelect(SelectTransformation<T> select) {
-		StreamTransformation<T> input = select.getInput();
-		Collection<Integer> resultIds = transform(input);
-
-
-		// the recursive transform might have already transformed this
-		if (alreadyTransformed.containsKey(select)) {
-			return alreadyTransformed.get(select);
-		}
-
-		List<Integer> virtualResultIds = new ArrayList<>();
-
-		for (int inputId : resultIds) {
-			int virtualId = StreamTransformation.getNewNodeId();
-			streamGraph.addVirtualSelectNode(inputId, virtualId, select.getSelectedNames());
-			virtualResultIds.add(virtualId);
-		}
-		return virtualResultIds;
-	}
-
-	/**
-	 * Transforms a {@code FeedbackTransformation}.
-	 *
-	 * <p>
-	 * This will recursively transform the input and the feedback edges. We return the concatenation
-	 * of the input IDs and the feedback IDs so that downstream operations can be wired to both.
-	 *
-	 * <p>
-	 * This is responsible for creating the IterationSource and IterationSink which
-	 * are used to feed back the elements.
-	 */
-	private <T> Collection<Integer> transformFeedback(FeedbackTransformation<T> iterate) {
-
-		if (iterate.getFeedbackEdges().size() <= 0) {
-			throw new IllegalStateException("Iteration " + iterate + " does not have any feedback edges.");
-		}
-
-		StreamTransformation<T> input = iterate.getInput();
-		List<Integer> resultIds = new ArrayList<>();
-
-		// first transform the input stream(s) and store the result IDs
-		resultIds.addAll(transform(input));
-
-		// the recursive transform might have already transformed this
-		if (alreadyTransformed.containsKey(iterate)) {
-			return alreadyTransformed.get(iterate);
-		}
-
-
-		// create the fake iteration source/sink pair
-		Tuple2<StreamNode, StreamNode> itSourceAndSink = streamGraph.createIterationSourceAndSink(
-				iterate.getId(),
-				getNewIterationNodeId(),
-				getNewIterationNodeId(),
-				iterate.getWaitTime(),
-				iterate.getParallelism());
-
-		StreamNode itSource = itSourceAndSink.f0;
-		StreamNode itSink = itSourceAndSink.f1;
-
-		// We set the proper serializers for the sink/source
-		streamGraph.setSerializers(itSource.getId(), null, null, iterate.getOutputType().createSerializer(env.getConfig()));
-		streamGraph.setSerializers(itSink.getId(), iterate.getOutputType().createSerializer(env.getConfig()), null, null);
-
-		// also add the feedback source ID to the result IDs, so that downstream operators will
-		// add both as input
-		resultIds.add(itSource.getId());
-
-		// at the iterate to the already-seen-set with the result IDs, so that we can transform
-		// the feedback edges and let them stop when encountering the iterate node
-		alreadyTransformed.put(iterate, resultIds);
-
-		for (StreamTransformation<T> feedbackEdge : iterate.getFeedbackEdges()) {
-			Collection<Integer> feedbackIds = transform(feedbackEdge);
-			for (Integer feedbackId: feedbackIds) {
-				streamGraph.addEdge(feedbackId,
-						itSink.getId(),
-						0
-				);
-			}
-		}
-
-		return resultIds;
-	}
-
-	/**
-	 * Transforms a {@code CoFeedbackTransformation}.
-	 *
-	 * <p>
-	 * This will only transform feedback edges, the result of this transform will be wired
-	 * to the second input of a Co-Transform. The original input is wired directly to the first
-	 * input of the downstream Co-Transform.
-	 *
-	 * <p>
-	 * This is responsible for creating the IterationSource and IterationSink which
-	 * are used to feed back the elements.
-	 */
-	private <F> Collection<Integer> transformCoFeedback(CoFeedbackTransformation<F> coIterate) {
-
-		// For Co-Iteration we don't need to transform the input and wire the input to the
-		// head operator by returning the input IDs, the input is directly wired to the left
-		// input of the co-operation. This transform only needs to return the ids of the feedback
-		// edges, since they need to be wired to the second input of the co-operation.
-
-		// create the fake iteration source/sink pair
-		Tuple2<StreamNode, StreamNode> itSourceAndSink = streamGraph.createIterationSourceAndSink(
-				coIterate.getId(),
-				getNewIterationNodeId(),
-				getNewIterationNodeId(),
-				coIterate.getWaitTime(),
-				coIterate.getParallelism());
-
-		StreamNode itSource = itSourceAndSink.f0;
-		StreamNode itSink = itSourceAndSink.f1;
-
-		// We set the proper serializers for the sink/source
-		streamGraph.setSerializers(itSource.getId(), null, null, coIterate.getOutputType().createSerializer(env.getConfig()));
-		streamGraph.setSerializers(itSink.getId(), coIterate.getOutputType().createSerializer(env.getConfig()), null, null);
-
-		Collection<Integer> resultIds = Collections.singleton(itSource.getId());
-
-		// at the iterate to the already-seen-set with the result IDs, so that we can transform
-		// the feedback edges and let them stop when encountering the iterate node
-		alreadyTransformed.put(coIterate, resultIds);
-
-		for (StreamTransformation<F> feedbackEdge : coIterate.getFeedbackEdges()) {
-			Collection<Integer> feedbackIds = transform(feedbackEdge);
-			for (Integer feedbackId: feedbackIds) {
-				streamGraph.addEdge(feedbackId,
-						itSink.getId(),
-						0
-				);
-			}
-		}
-
-		return Collections.singleton(itSource.getId());
-	}
-
-	/**
-	 * Transforms a {@code SourceTransformation}.
-	 */
-	private <T> Collection<Integer> transformSource(SourceTransformation<T> source) {
-		streamGraph.addSource(source.getId(),
-				source.getOperator(),
-				null,
-				source.getOutputType(),
-				"Source: " + source.getName());
-		if (source.getOperator().getUserFunction() instanceof FileSourceFunction) {
-			FileSourceFunction<T> fs = (FileSourceFunction<T>) source.getOperator().getUserFunction();
-			streamGraph.setInputFormat(source.getId(), fs.getFormat());
-		}
-		streamGraph.setParallelism(source.getId(), source.getParallelism());
-		return Collections.singleton(source.getId());
-	}
-
-	/**
-	 * Transforms a {@code SourceTransformation}.
-	 */
-	private <T> Collection<Integer> transformSink(SinkTransformation<T> sink) {
-
-		Collection<Integer> inputIds = transform(sink.getInput());
-
-		streamGraph.addSink(sink.getId(),
-				sink.getOperator(),
-				sink.getInput().getOutputType(),
-				null,
-				"Sink: " + sink.getName());
-
-		streamGraph.setParallelism(sink.getId(), sink.getParallelism());
-
-		for (Integer inputId: inputIds) {
-			streamGraph.addEdge(inputId,
-					sink.getId(),
-					0
-			);
-		}
-
-
-		if (sink.getStateKeySelector() != null) {
-			TypeSerializer<?> keySerializer = sink.getStateKeyType().createSerializer(env.getConfig());
-			streamGraph.setKey(sink.getId(), sink.getStateKeySelector(), keySerializer);
-		}
-
-		return Collections.emptyList();
-	}
-
-	/**
-	 * Transforms a {@code OneInputTransformation}.
-	 *
-	 * <p>
-	 * This recusively transforms the inputs, creates a new {@code StreamNode} in the graph and
-	 * wired the inputs to this new node.
-	 */
-	private <IN, OUT> Collection<Integer> transformOnInputTransform(OneInputTransformation<IN, OUT> transform) {
-
-		Collection<Integer> inputIds = transform(transform.getInput());
-
-		// the recursive call might have already transformed this
-		if (alreadyTransformed.containsKey(transform)) {
-			return alreadyTransformed.get(transform);
-		}
-
-		streamGraph.addOperator(transform.getId(),
-				transform.getOperator(),
-				transform.getInputType(),
-				transform.getOutputType(),
-				transform.getName());
-
-		if (transform.getStateKeySelector() != null) {
-			TypeSerializer<?> keySerializer = transform.getStateKeyType().createSerializer(env.getConfig());
-			streamGraph.setKey(transform.getId(), transform.getStateKeySelector(), keySerializer);
-		}
-		if (transform.getStateKeyType() != null) {
-			
-		}
-
-		streamGraph.setParallelism(transform.getId(), transform.getParallelism());
-
-		for (Integer inputId: inputIds) {
-			streamGraph.addEdge(inputId, transform.getId(), 0);
-		}
-
-		return Collections.singleton(transform.getId());
-	}
-
-	/**
-	 * Transforms a {@code TwoInputTransformation}.
-	 *
-	 * <p>
-	 * This recusively transforms the inputs, creates a new {@code StreamNode} in the graph and
-	 * wired the inputs to this new node.
-	 */
-	private <IN1, IN2, OUT> Collection<Integer> transformTwoInputTransform(TwoInputTransformation<IN1, IN2, OUT> transform) {
-
-		Collection<Integer> inputIds1 = transform(transform.getInput1());
-		Collection<Integer> inputIds2 = transform(transform.getInput2());
-
-		// the recursive call might have already transformed this
-		if (alreadyTransformed.containsKey(transform)) {
-			return alreadyTransformed.get(transform);
-		}
-
-		streamGraph.addCoOperator(
-				transform.getId(),
-				transform.getOperator(),
-				transform.getInputType1(),
-				transform.getInputType2(),
-				transform.getOutputType(),
-				transform.getName());
-
-		streamGraph.setParallelism(transform.getId(), transform.getParallelism());
-
-		for (Integer inputId: inputIds1) {
-			streamGraph.addEdge(inputId,
-					transform.getId(),
-					1
-			);
-		}
-
-		for (Integer inputId: inputIds2) {
-			streamGraph.addEdge(inputId,
-					transform.getId(),
-					2
-			);
-		}
-
-		return Collections.singleton(transform.getId());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
deleted file mode 100644
index 608e648..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.graph;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
-import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapperFactory;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-
-/**
- * Class representing the operators in the streaming programs, with all their
- * properties.
- * 
- */
-public class StreamNode implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-	private static int currentSlotSharingIndex = 1;
-
-	transient private StreamExecutionEnvironment env;
-
-	private Integer id;
-	private Integer parallelism = null;
-	private Long bufferTimeout = null;
-	private String operatorName;
-	private Integer slotSharingID;
-	private boolean isolatedSlot = false;
-	private KeySelector<?,?> statePartitioner;
-	private TypeSerializer<?> stateKeySerializer;
-
-	private transient StreamOperator<?> operator;
-	private List<OutputSelector<?>> outputSelectors;
-	private TypeSerializer<?> typeSerializerIn1;
-	private TypeSerializer<?> typeSerializerIn2;
-	private TypeSerializer<?> typeSerializerOut;
-
-	private List<StreamEdge> inEdges = new ArrayList<StreamEdge>();
-	private List<StreamEdge> outEdges = new ArrayList<StreamEdge>();
-
-	private Class<? extends AbstractInvokable> jobVertexClass;
-
-	private InputFormat<?, ?> inputFormat;
-
-	public StreamNode(StreamExecutionEnvironment env, Integer id, StreamOperator<?> operator,
-			String operatorName, List<OutputSelector<?>> outputSelector,
-			Class<? extends AbstractInvokable> jobVertexClass) {
-		this.env = env;
-		this.id = id;
-		this.operatorName = operatorName;
-		this.operator = operator;
-		this.outputSelectors = outputSelector;
-		this.jobVertexClass = jobVertexClass;
-		this.slotSharingID = currentSlotSharingIndex;
-	}
-
-	public void addInEdge(StreamEdge inEdge) {
-		if (inEdge.getTargetId() != getId()) {
-			throw new IllegalArgumentException("Destination id doesn't match the StreamNode id");
-		} else {
-			inEdges.add(inEdge);
-		}
-	}
-
-	public void addOutEdge(StreamEdge outEdge) {
-		if (outEdge.getSourceId() != getId()) {
-			throw new IllegalArgumentException("Source id doesn't match the StreamNode id");
-		} else {
-			outEdges.add(outEdge);
-		}
-	}
-
-	public List<StreamEdge> getOutEdges() {
-		return outEdges;
-	}
-
-	public List<StreamEdge> getInEdges() {
-		return inEdges;
-	}
-
-	public List<Integer> getOutEdgeIndices() {
-		List<Integer> outEdgeIndices = new ArrayList<Integer>();
-
-		for (StreamEdge edge : outEdges) {
-			outEdgeIndices.add(edge.getTargetId());
-		}
-
-		return outEdgeIndices;
-	}
-
-	public List<Integer> getInEdgeIndices() {
-		List<Integer> inEdgeIndices = new ArrayList<Integer>();
-
-		for (StreamEdge edge : inEdges) {
-			inEdgeIndices.add(edge.getSourceId());
-		}
-
-		return inEdgeIndices;
-	}
-
-	public Integer getId() {
-		return id;
-	}
-
-	public int getParallelism() {
-		if (parallelism == -1) {
-			return env.getParallelism();
-		} else {
-			return parallelism;
-		}
-	}
-
-	public void setParallelism(Integer parallelism) {
-		this.parallelism = parallelism;
-	}
-
-	public Long getBufferTimeout() {
-		return bufferTimeout != null ? bufferTimeout : env.getBufferTimeout();
-	}
-
-	public void setBufferTimeout(Long bufferTimeout) {
-		this.bufferTimeout = bufferTimeout;
-	}
-
-	public StreamOperator<?> getOperator() {
-		return operator;
-	}
-
-	public void setOperator(StreamOperator<?> operator) {
-		this.operator = operator;
-	}
-
-	public String getOperatorName() {
-		return operatorName;
-	}
-
-	public void setOperatorName(String operatorName) {
-		this.operatorName = operatorName;
-	}
-
-	public List<OutputSelector<?>> getOutputSelectors() {
-		return outputSelectors;
-	}
-
-	public OutputSelectorWrapper<?> getOutputSelectorWrapper() {
-		return OutputSelectorWrapperFactory.create(getOutputSelectors());
-	}
-
-	public void addOutputSelector(OutputSelector<?> outputSelector) {
-		this.outputSelectors.add(outputSelector);
-	}
-
-	public TypeSerializer<?> getTypeSerializerIn1() {
-		return typeSerializerIn1;
-	}
-
-	public void setSerializerIn1(TypeSerializer<?> typeSerializerIn1) {
-		this.typeSerializerIn1 = typeSerializerIn1;
-	}
-
-	public TypeSerializer<?> getTypeSerializerIn2() {
-		return typeSerializerIn2;
-	}
-
-	public void setSerializerIn2(TypeSerializer<?> typeSerializerIn2) {
-		this.typeSerializerIn2 = typeSerializerIn2;
-	}
-
-	public TypeSerializer<?> getTypeSerializerOut() {
-		return typeSerializerOut;
-	}
-
-	public void setSerializerOut(TypeSerializer<?> typeSerializerOut) {
-		this.typeSerializerOut = typeSerializerOut;
-	}
-
-	public Class<? extends AbstractInvokable> getJobVertexClass() {
-		return jobVertexClass;
-	}
-
-	public InputFormat<?, ?> getInputFormat() {
-		return inputFormat;
-	}
-
-	public void setInputFormat(InputFormat<?, ?> inputFormat) {
-		this.inputFormat = inputFormat;
-	}
-
-	public int getSlotSharingID() {
-		return isolatedSlot ? -1 : slotSharingID;
-	}
-
-	public void startNewSlotSharingGroup() {
-		this.slotSharingID = ++currentSlotSharingIndex;
-	}
-
-	public void isolateSlot() {
-		isolatedSlot = true;
-	}
-	
-	@Override
-	public String toString() {
-		return operatorName + "-" + id;
-	}
-
-	public KeySelector<?, ?> getStatePartitioner() {
-		return statePartitioner;
-	}
-
-	public void setStatePartitioner(KeySelector<?, ?> statePartitioner) {
-		this.statePartitioner = statePartitioner;
-	}
-
-	public TypeSerializer<?> getStateKeySerializer() {
-		return stateKeySerializer;
-	}
-
-	public void setStateKeySerializer(TypeSerializer<?> stateKeySerializer) {
-		this.stateKeySerializer = stateKeySerializer;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		StreamNode that = (StreamNode) o;
-
-		return id.equals(that.id);
-	}
-
-	@Override
-	public int hashCode() {
-		return id.hashCode();
-	}
-}


[27/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
deleted file mode 100644
index 4c0f59f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java
+++ /dev/null
@@ -1,152 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Test;
-
-@SuppressWarnings("serial")
-public class TypeFillTest extends StreamingMultipleProgramsTestBase {
-
-	@Test
-	public void test() {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		try {
-			env.addSource(new TestSource<Integer>()).print();
-			fail();
-		} catch (Exception e) {
-		}
-
-		DataStream<Long> source = env.generateSequence(1, 10);
-
-		try {
-			source.map(new TestMap<Long, Long>()).print();
-			fail();
-		} catch (Exception e) {
-		}
-		try {
-			source.flatMap(new TestFlatMap<Long, Long>()).print();
-			fail();
-		} catch (Exception e) {
-		}
-		try {
-			source.connect(source).map(new TestCoMap<Long, Long, Integer>()).print();
-			fail();
-		} catch (Exception e) {
-		}
-		try {
-			source.connect(source).flatMap(new TestCoFlatMap<Long, Long, Integer>()).print();
-			fail();
-		} catch (Exception e) {
-		}
-
-		env.addSource(new TestSource<Integer>()).returns("Integer");
-		source.map(new TestMap<Long, Long>()).returns(Long.class).print();
-		source.flatMap(new TestFlatMap<Long, Long>()).returns("Long").print();
-		source.connect(source).map(new TestCoMap<Long, Long, Integer>()).returns("Integer").print();
-		source.connect(source).flatMap(new TestCoFlatMap<Long, Long, Integer>())
-				.returns(BasicTypeInfo.INT_TYPE_INFO).print();
-		
-		assertEquals(BasicTypeInfo.LONG_TYPE_INFO,
-				source.map(new TestMap<Long, Long>()).returns(Long.class).getType());
-
-		SingleOutputStreamOperator<String, ?> map = source.map(new MapFunction<Long, String>() {
-
-			@Override
-			public String map(Long value) throws Exception {
-				return null;
-			}
-		});
-
-		map.print();
-		try {
-			map.returns("String");
-			fail();
-		} catch (Exception e) {
-		}
-
-	}
-
-	private class TestSource<T> implements SourceFunction<T> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void run(SourceContext<T> ctx) throws Exception {
-
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-
-	private class TestMap<T, O> implements MapFunction<T, O> {
-		@Override
-		public O map(T value) throws Exception {
-			return null;
-		}
-	}
-
-	private class TestFlatMap<T, O> implements FlatMapFunction<T, O> {
-		@Override
-		public void flatMap(T value, Collector<O> out) throws Exception {
-		}
-	}
-
-	private class TestCoMap<IN1, IN2, OUT> implements CoMapFunction<IN1, IN2, OUT> {
-
-		@Override
-		public OUT map1(IN1 value) {
-			return null;
-		}
-
-		@Override
-		public OUT map2(IN2 value) {
-			return null;
-		}
-
-	}
-
-	private class TestCoFlatMap<IN1, IN2, OUT> implements CoFlatMapFunction<IN1, IN2, OUT> {
-
-		@Override
-		public void flatMap1(IN1 value, Collector<OUT> out) throws Exception {
-		}
-
-		@Override
-		public void flatMap2(IN2 value, Collector<OUT> out) throws Exception {
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
deleted file mode 100644
index d2e24c9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/DirectedOutputTest.java
+++ /dev/null
@@ -1,119 +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.collector;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestListResultSink;
-import org.junit.Test;
-
-public class DirectedOutputTest extends StreamingMultipleProgramsTestBase {
-
-	private static final String TEN = "ten";
-	private static final String ODD = "odd";
-	private static final String EVEN = "even";
-	private static final String NON_SELECTED = "nonSelected";
-
-	static final class MyOutputSelector implements OutputSelector<Long> {
-		private static final long serialVersionUID = 1L;
-
-		List<String> outputs = new ArrayList<String>();
-
-		@Override
-		public Iterable<String> select(Long value) {
-			outputs.clear();
-			if (value % 2 == 0) {
-				outputs.add(EVEN);
-			} else {
-				outputs.add(ODD);
-			}
-
-			if (value == 10L) {
-				outputs.add(TEN);
-			}
-
-			if (value == 11L) {
-				outputs.add(NON_SELECTED);
-			}
-			return outputs;
-		}
-	}
-
-	static final class ListSink implements SinkFunction<Long> {
-		private static final long serialVersionUID = 1L;
-
-		private String name;
-		private transient List<Long> list;
-
-		public ListSink(String name) {
-			this.name = name;
-		}
-
-		@Override
-		public void invoke(Long value) {
-			list.add(value);
-		}
-
-		private void readObject(java.io.ObjectInputStream in) throws IOException,
-				ClassNotFoundException {
-			in.defaultReadObject();
-			outputs.put(name, new ArrayList<Long>());
-			this.list = outputs.get(name);
-		}
-
-	}
-
-	private static Map<String, List<Long>> outputs = new HashMap<String, List<Long>>();
-
-	@Test
-	public void outputSelectorTest() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(3);
-
-		TestListResultSink<Long> evenSink = new TestListResultSink<Long>();
-		TestListResultSink<Long> oddAndTenSink = new TestListResultSink<Long>();
-		TestListResultSink<Long> evenAndOddSink = new TestListResultSink<Long>();
-		TestListResultSink<Long> allSink = new TestListResultSink<Long>();
-
-		SplitStream<Long> source = env.generateSequence(1, 11).split(new MyOutputSelector());
-		source.select(EVEN).addSink(evenSink);
-		source.select(ODD, TEN).addSink(oddAndTenSink);
-		source.select(EVEN, ODD).addSink(evenAndOddSink);
-		source.addSink(allSink);
-
-		env.execute();
-		assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), evenSink.getSortedResult());
-		assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L, 11L), oddAndTenSink.getSortedResult());
-		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L),
-				evenAndOddSink.getSortedResult());
-		assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L),
-				allSink.getSortedResult());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java
deleted file mode 100644
index a3d89f2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/collector/OutputSelectorTest.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.collector;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.junit.Test;
-
-public class OutputSelectorTest {
-
-	static final class MyOutputSelector implements OutputSelector<Tuple1<Integer>> {
-
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Iterable<String> select(Tuple1<Integer> tuple) {
-
-			String[] outputs = new String[tuple.f0];
-
-			for (Integer i = 0; i < tuple.f0; i++) {
-				outputs[i] = i.toString();
-			}
-			return Arrays.asList(outputs);
-		}
-	}
-
-	@Test
-	public void testGetOutputs() {
-		OutputSelector<Tuple1<Integer>> selector = new MyOutputSelector();
-		List<String> expectedOutputs = new ArrayList<String>();
-		expectedOutputs.add("0");
-		expectedOutputs.add("1");
-		assertEquals(expectedOutputs, selector.select(new Tuple1<Integer>(2)));
-		expectedOutputs.add("2");
-		assertEquals(expectedOutputs, selector.select(new Tuple1<Integer>(3)));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
deleted file mode 100644
index 020dda3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/complex/ComplexIntegrationTest.java
+++ /dev/null
@@ -1,837 +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.complex;
-
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.datastream.IterativeStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.co.CoMapFunction;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction;
-import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-@SuppressWarnings("serial")
-public class ComplexIntegrationTest extends StreamingMultipleProgramsTestBase {
-
-	// *************************************************************************
-	// GENERAL SETUP
-	// *************************************************************************
-
-	private String resultPath1;
-	private String resultPath2;
-	private String expected1;
-	private String expected2;
-	
-
-	@Rule
-	public TemporaryFolder tempFolder = new TemporaryFolder();
-
-	@Before
-	public void before() throws Exception {
-		resultPath1 = tempFolder.newFile().toURI().toString();
-		resultPath2 = tempFolder.newFile().toURI().toString();
-		expected1 = "";
-		expected2 = "";
-	}
-
-	@After
-	public void after() throws Exception {
-		compareResultsByLinesInMemory(expected1, resultPath1);
-		compareResultsByLinesInMemory(expected2, resultPath2);
-	}
-
-	// *************************************************************************
-	// INTEGRATION TESTS
-	// *************************************************************************
-
-	@Test
-	public void complexIntegrationTest1() throws Exception {
-		//Testing data stream splitting with tuples
-
-		expected1 = "";
-		for (int i = 0; i < 8; i++) {
-			expected1 += "(10,(a,1))\n";
-		}
-		//i == 8
-		expected1 += "(10,(a,1))";
-
-		expected2 = "";
-		for (int i = 0; i < 18; i++) {
-			expected2 += "(20,(a,1))\n";
-		}
-		//i == 18
-		expected2 += "(20,(a,1))";
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		DataStream<Tuple2<Long, Tuple2<String, Long>>> sourceStream1 = env.addSource(new TupleSource()).setParallelism(1);
-
-		IterativeStream<Tuple2<Long, Tuple2<String, Long>>> it = sourceStream1.map(new MapFunction<Tuple2<Long, Tuple2<String, Long>>,Tuple2<Long, Tuple2<String, Long>>>(){
-
-					Tuple2<Long, Tuple2<String, Long>> result = new Tuple2<>(
-							0L, new Tuple2<>("", 0L));
-
-					@Override
-					public Tuple2<Long, Tuple2<String, Long>> map(
-							Tuple2<Long, Tuple2<String, Long>> value) throws Exception {
-						result.f0 = result.f0 + value.f0;
-						result.f1 = value.f1;
-						return result;
-			}
-			
-		})
-				.setParallelism(1).filter(new FilterFunction
-				<Tuple2<Long, Tuple2<String, Long>>>() {
-
-			@Override
-			public boolean filter(Tuple2<Long, Tuple2<String, Long>> value) throws Exception {
-				return value.f0 < 20;
-			}
-		}).iterate(5000);
-
-		SplitStream<Tuple2<Long, Tuple2<String, Long>>> step = it.map(new IncrementMap()).split(new
-				MyOutputSelector());
-		it.closeWith(step.select("iterate"));
-
-		step.select("firstOutput")
-				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-		step.select("secondOutput")
-				.writeAsText(resultPath2, FileSystem.WriteMode.OVERWRITE);
-
-		env.execute();
-	}
-
-	// Disabled, because it depends on strange behaviour, for example of the sum() function.
-	// This test evens fails, for example, if the order of only two lines in the "input" is changed.
-	@SuppressWarnings("unchecked")
-	@Ignore
-	@Test
-	public void complexIntegrationTest2() throws Exception {
-		//Testing POJO source, grouping by multiple filds and windowing with timestamp
-
-		expected1 = "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" +
-				"water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" +
-				"water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" +
-				"water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" + "water_melon-b\n" +
-				"water_melon-b\n" + "orange-b\n" + "orange-b\n" + "orange-b\n" + "orange-b\n" + "orange-b\n" +
-				"orange-b\n" + "orange-c\n" + "orange-c\n" + "orange-c\n" + "orange-c\n" + "orange-d\n" + "orange-d\n" +
-				"peach-d\n" + "peach-d\n";
-
-		List<Tuple5<Integer, String, Character, Double, Boolean>> input = Arrays.asList(
-				new Tuple5<>(1, "apple", 'j', 0.1, false),
-				new Tuple5<>(1, "peach", 'b', 0.8, false),
-				new Tuple5<>(1, "orange", 'c', 0.7, true),
-				new Tuple5<>(2, "apple", 'd', 0.5, false),
-				new Tuple5<>(2, "peach", 'j', 0.6, false),
-				new Tuple5<>(3, "orange", 'b', 0.2, true),
-				new Tuple5<>(6, "apple", 'c', 0.1, false),
-				new Tuple5<>(7, "peach", 'd', 0.4, false),
-				new Tuple5<>(8, "orange", 'j', 0.2, true),
-				new Tuple5<>(10, "apple", 'b', 0.1, false),
-				new Tuple5<>(10, "peach", 'c', 0.5, false),
-				new Tuple5<>(11, "orange", 'd', 0.3, true),
-				new Tuple5<>(11, "apple", 'j', 0.3, false),
-				new Tuple5<>(12, "peach", 'b', 0.9, false),
-				new Tuple5<>(13, "orange", 'c', 0.7, true),
-				new Tuple5<>(15, "apple", 'd', 0.2, false),
-				new Tuple5<>(16, "peach", 'j', 0.8, false),
-				new Tuple5<>(16, "orange", 'b', 0.8, true),
-				new Tuple5<>(16, "apple", 'c', 0.1, false),
-				new Tuple5<>(17, "peach", 'd', 1.0, true));
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableTimestamps();
-
-		SingleOutputStreamOperator<Tuple5<Integer, String, Character, Double, Boolean>, DataStreamSource<Tuple5<Integer, String, Character, Double, Boolean>>> sourceStream21 = env.fromCollection(input);
-		DataStream<OuterPojo> sourceStream22 = env.addSource(new PojoSource());
-
-		sourceStream21
-				.assignTimestamps(new MyTimestampExtractor())
-				.keyBy(2, 2)
-				.timeWindow(Time.of(10, TimeUnit.MILLISECONDS), Time.of(4, TimeUnit.MILLISECONDS))
-				.maxBy(3)
-				.map(new MyMapFunction2())
-				.flatMap(new MyFlatMapFunction())
-				.connect(sourceStream22)
-				.map(new MyCoMapFunction())
-				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-
-		env.execute();
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void complexIntegrationTest3() throws Exception {
-		//Heavy prime factorisation with maps and flatmaps
-
-		expected1 = "541\n" + "1223\n" + "3319\n" + "5851\n" + "1987\n" + "8387\n" + "15907\n" + "10939\n" +
-				"4127\n" + "2477\n" + "6737\n" + "13421\n" + "4987\n" + "4999\n" + "18451\n" + "9283\n" + "7499\n" +
-				"16937\n" + "11927\n" + "9973\n" + "14431\n" + "19507\n" + "12497\n" + "17497\n" + "14983\n" +
-				"19997\n";
-
-		expected1 = "541\n" + "1223\n" + "1987\n" + "2741\n" + "3571\n" + "10939\n" + "4409\n" +
-				"5279\n" + "11927\n" + "6133\n" + "6997\n" + "12823\n" + "7919\n" + "8831\n" +
-				"13763\n" + "9733\n" + "9973\n" + "14759\n" + "15671\n" + "16673\n" + "17659\n" +
-				"18617\n" + "19697\n" + "19997\n";
-
-		for (int i = 2; i < 100; i++) {
-			expected2 += "(" + i + "," + 20000 / i + ")\n";
-		}
-		for (int i = 19901; i < 20000; i++) {
-			expected2 += "(" + i + "," + 20000 / i + ")\n";
-		}
-		//i == 20000
-		expected2 += "(" + 20000 + "," + 1 + ")";
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// set to parallelism 1 because otherwise we don't know which elements go to which parallel
-		// count-window.
-		env.setParallelism(1);
-
-		env.setBufferTimeout(0);
-
-		DataStream<Long> sourceStream31 = env.generateSequence(1, 10000);
-		DataStream<Long> sourceStream32 = env.generateSequence(10001, 20000);
-
-		sourceStream31.filter(new PrimeFilterFunction())
-				.windowAll(GlobalWindows.create())
-				.trigger(PurgingTrigger.of(CountTrigger.of(100)))
-				.max(0)
-				.union(sourceStream32.filter(new PrimeFilterFunction())
-						.windowAll(GlobalWindows.create())
-						.trigger(PurgingTrigger.of(CountTrigger.of(100)))
-						.max(0))
-				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-
-		sourceStream31.flatMap(new DivisorsFlatMapFunction())
-				.union(sourceStream32.flatMap(new DivisorsFlatMapFunction())).map(new MapFunction<Long, Tuple2<Long,
-				Integer>>() {
-
-			@Override
-			public Tuple2<Long, Integer> map(Long value) throws Exception {
-				return new Tuple2<>(value, 1);
-			}
-		})
-				.keyBy(0)
-				.window(GlobalWindows.create())
-				.trigger(PurgingTrigger.of(CountTrigger.of(10_000)))
-				.sum(1)
-				.filter(new FilterFunction<Tuple2<Long, Integer>>() {
-
-					@Override
-					public boolean filter(Tuple2<Long, Integer> value) throws Exception {
-						return value.f0 < 100 || value.f0 > 19900;
-					}
-				})
-				.writeAsText(resultPath2, FileSystem.WriteMode.OVERWRITE);
-
-		env.execute();
-	}
-
-	@Test
-	@Ignore
-	public void complexIntegrationTest4() throws Exception {
-		//Testing mapping and delta-policy windowing with custom class
-
-		expected1 = "((100,100),0)\n" + "((120,122),5)\n" + "((121,125),6)\n" + "((138,144),9)\n" +
-				"((139,147),10)\n" + "((156,166),13)\n" + "((157,169),14)\n" + "((174,188),17)\n" + "((175,191),18)\n" +
-				"((192,210),21)\n" + "((193,213),22)\n" + "((210,232),25)\n" + "((211,235),26)\n" + "((228,254),29)\n" +
-				"((229,257),30)\n" + "((246,276),33)\n" + "((247,279),34)\n" + "((264,298),37)\n" + "((265,301),38)\n" +
-				"((282,320),41)\n" + "((283,323),42)\n" + "((300,342),45)\n" + "((301,345),46)\n" + "((318,364),49)\n" +
-				"((319,367),50)\n" + "((336,386),53)\n" + "((337,389),54)\n" + "((354,408),57)\n" + "((355,411),58)\n" +
-				"((372,430),61)\n" + "((373,433),62)\n" + "((390,452),65)\n" + "((391,455),66)\n" + "((408,474),69)\n" +
-				"((409,477),70)\n" + "((426,496),73)\n" + "((427,499),74)\n" + "((444,518),77)\n" + "((445,521),78)\n" +
-				"((462,540),81)\n" + "((463,543),82)\n" + "((480,562),85)\n" + "((481,565),86)\n" + "((498,584),89)\n" +
-				"((499,587),90)\n" + "((516,606),93)\n" + "((517,609),94)\n" + "((534,628),97)\n" + "((535,631),98)";
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setParallelism(1);
-
-		env.addSource(new RectangleSource())
-				.global()
-				.map(new RectangleMapFunction())
-				.windowAll(GlobalWindows.create())
-				.trigger(PurgingTrigger.of(DeltaTrigger.of(0.0, new MyDelta())))
-				.apply(new MyWindowMapFunction())
-				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-
-		env.execute();
-	}
-
-	private static class MyDelta implements DeltaFunction<Tuple2<Rectangle, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public double getDelta(Tuple2<Rectangle, Integer> oldDataPoint, Tuple2<Rectangle,
-				Integer> newDataPoint) {
-			return (newDataPoint.f0.b - newDataPoint.f0.a) - (oldDataPoint.f0.b - oldDataPoint.f0.a);
-		}
-	}
-
-
-	@Test
-	public void complexIntegrationTest5() throws Exception {
-		//Turning on and off chaining
-
-		expected1 = "1\n" + "2\n" + "2\n" + "3\n" + "3\n" + "3\n" + "4\n" + "4\n" + "4\n" + "4\n" + "5\n" + "5\n" +
-				"5\n" + "5\n" + "5\n";
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// Set to parallelism 1 to make it deterministic, otherwise, it is not clear which
-		// elements will go to which parallel instance of the fold
-		env.setParallelism(1);
-		
-		env.setBufferTimeout(0);
-
-		DataStream<Long> dataStream51 = env.generateSequence(1, 5)
-				.map(new MapFunction<Long, Long>() {
-
-					@Override
-					public Long map(Long value) throws Exception {
-						return value;
-					}
-				}).startNewChain()
-				.filter(new FilterFunction<Long>() {
-
-					@Override
-					public boolean filter(Long value) throws Exception {
-						return true;
-					}
-				}).disableChaining()
-				.flatMap(new SquareFlatMapFunction());
-
-		DataStream<Long> dataStream53 = dataStream51.map(new MapFunction<Long, Long>() {
-
-			@Override
-			public Long map(Long value) throws Exception {
-				return value;
-			}
-		});
-
-
-		dataStream53.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-
-		env.execute();
-	}
-
-
-	@Test
-	@Ignore
-	public void complexIntegrationTest6() throws Exception {
-		//Testing java collections and date-time types
-
-		expected1 = "(6,(a,6))\n" + "(6,(b,3))\n" + "(6,(c,4))\n" + "(6,(d,2))\n" + "(6,(f,2))\n" +
-				"(7,(a,1))\n" + "(7,(b,2))\n" + "(7,(c,3))\n" + "(7,(d,1))\n" + "(7,(e,1))\n" + "(7,(f,1))\n" +
-				"(8,(a,6))\n" + "(8,(b,4))\n" + "(8,(c,5))\n" + "(8,(d,1))\n" + "(8,(e,2))\n" + "(8,(f,2))\n" +
-				"(9,(a,4))\n" + "(9,(b,4))\n" + "(9,(c,7))\n" + "(9,(d,3))\n" + "(9,(e,1))\n" + "(9,(f,2))\n" +
-				"(10,(a,3))\n" + "(10,(b,2))\n" + "(10,(c,3))\n" + "(10,(d,2))\n" + "(10,(e,1))\n" + "(10,(f,1))";
-		expected2 = "[a, a, c, c, d, f]\n" + "[a, b, b, d]\n" + "[a, a, a, b, c, c, f]\n" + "[a, d, e]\n" +
-				"[b, b, c, c, c, f]\n" + "[a, a, a, a, b, b, c, c, e]\n" + "[a, a, b, b, c, c, c, d, e, f, f]\n" +
-				"[a, a, a, b, c, c, c, d, d, f]\n" + "[a, b, b, b, c, c, c, c, d, e, f]\n" +
-				"[a, a, a, b, b, c, c, c, d, d, e, f]";
-
-		SimpleDateFormat ft = new SimpleDateFormat("dd-MM-yyyy");
-
-		ArrayList<Tuple2<Date, HashMap<Character, Integer>>> sales = new ArrayList<>();
-		HashMap<Character, Integer> sale1 = new HashMap<>();
-		sale1.put('a', 2);
-		sale1.put('c', 2);
-		sale1.put('d', 1);
-		sale1.put('f', 1);
-		sales.add(new Tuple2<>(ft.parse("03-06-2014"), sale1));
-
-		HashMap<Character, Integer> sale2 = new HashMap<>();
-		sale2.put('a', 1);
-		sale2.put('b', 2);
-		sale2.put('d', 1);
-		sales.add(new Tuple2<>(ft.parse("10-06-2014"), sale2));
-
-		HashMap<Character, Integer> sale3 = new HashMap<>();
-		sale3.put('a', 3);
-		sale3.put('b', 1);
-		sale3.put('c', 2);
-		sale3.put('f', 1);
-		sales.add(new Tuple2<>(ft.parse("29-06-2014"), sale3));
-
-		HashMap<Character, Integer> sale4 = new HashMap<>();
-		sale4.put('a', 1);
-		sale4.put('d', 1);
-		sale4.put('e', 1);
-		sales.add(new Tuple2<>(ft.parse("15-07-2014"), sale4));
-
-		HashMap<Character, Integer> sale5 = new HashMap<>();
-		sale5.put('b', 2);
-		sale5.put('c', 3);
-		sale5.put('f', 1);
-		sales.add(new Tuple2<>(ft.parse("24-07-2014"), sale5));
-
-		HashMap<Character, Integer> sale6 = new HashMap<>();
-		sale6.put('a', 4);
-		sale6.put('b', 2);
-		sale6.put('c', 2);
-		sale6.put('e', 1);
-		sales.add(new Tuple2<>(ft.parse("17-08-2014"), sale6));
-
-		HashMap<Character, Integer> sale7 = new HashMap<>();
-		sale7.put('a', 2);
-		sale7.put('b', 2);
-		sale7.put('c', 3);
-		sale7.put('d', 1);
-		sale7.put('e', 1);
-		sale7.put('f', 2);
-		sales.add(new Tuple2<>(ft.parse("27-08-2014"), sale7));
-
-		HashMap<Character, Integer> sale8 = new HashMap<>();
-		sale8.put('a', 3);
-		sale8.put('b', 1);
-		sale8.put('c', 3);
-		sale8.put('d', 2);
-		sale8.put('f', 1);
-		sales.add(new Tuple2<>(ft.parse("16-09-2014"), sale8));
-
-		HashMap<Character, Integer> sale9 = new HashMap<>();
-		sale9.put('a', 1);
-		sale9.put('b', 3);
-		sale9.put('c', 4);
-		sale9.put('d', 1);
-		sale9.put('e', 1);
-		sale9.put('f', 1);
-		sales.add(new Tuple2<>(ft.parse("25-09-2014"), sale9));
-
-		HashMap<Character, Integer> sale10 = new HashMap<>();
-		sale10.put('a', 3);
-		sale10.put('b', 2);
-		sale10.put('c', 3);
-		sale10.put('d', 2);
-		sale10.put('e', 1);
-		sale10.put('f', 1);
-		sales.add(new Tuple2<>(ft.parse("01-10-2014"), sale10));
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.getConfig().enableTimestamps();
-
-		DataStream<Tuple2<Date, HashMap<Character, Integer>>> sourceStream6 = env.fromCollection(sales);
-		sourceStream6
-				.assignTimestamps(new Timestamp6())
-				.timeWindowAll(Time.of(1, TimeUnit.MILLISECONDS))
-				.reduce(new SalesReduceFunction())
-				.flatMap(new FlatMapFunction6())
-				.writeAsText(resultPath1, FileSystem.WriteMode.OVERWRITE);
-
-		sourceStream6.map(new MapFunction6())
-				.writeAsText(resultPath2, FileSystem.WriteMode.OVERWRITE);
-
-		env.execute();
-
-	}
-
-	// *************************************************************************
-	// FUNCTIONS
-	// *************************************************************************
-
-	private static class MyMapFunction2 implements MapFunction<Tuple5<Integer, String, Character, Double, Boolean>, Tuple4<Integer, String,
-			Double, Boolean>> {
-
-		@Override
-		public Tuple4<Integer, String, Double, Boolean> map(Tuple5<Integer, String, Character, Double,
-				Boolean> value) throws Exception {
-			return new Tuple4<>(value.f0, value.f1 + "-" + value.f2,
-					value.f3, value.f4);
-		}
-
-	}
-
-	private static class PojoSource implements SourceFunction<OuterPojo> {
-		private static final long serialVersionUID = 1L;
-
-		long cnt = 0;
-
-		@Override
-		public void run(SourceContext<OuterPojo> ctx) throws Exception {
-			for (int i = 0; i < 20; i++) {
-				OuterPojo result = new OuterPojo(new InnerPojo(cnt / 2, "water_melon-b"), 2L);
-				ctx.collect(result);
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-
-	private static class TupleSource implements SourceFunction<Tuple2<Long, Tuple2<String, Long>>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void run(SourceContext<Tuple2<Long, Tuple2<String, Long>>> ctx) throws Exception {
-			for (int i = 0; i < 20; i++) {
-				Tuple2<Long, Tuple2<String, Long>> result = new Tuple2<>(1L, new Tuple2<>("a", 1L));
-				ctx.collect(result);
-			}
-		}
-
-		@Override
-		public void cancel() {
-
-		}
-	}
-
-	private class IncrementMap implements MapFunction<Tuple2<Long, Tuple2<String, Long>>, Tuple2<Long, Tuple2<String,
-			Long>>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Long, Tuple2<String, Long>> map(Tuple2<Long, Tuple2<String, Long>> value) throws Exception {
-			return new Tuple2<>(value.f0 + 1, value.f1);
-		}
-	}
-
-	private static class MyTimestampExtractor implements TimestampExtractor<Tuple5<Integer, String, Character, Double, Boolean>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple5<Integer, String, Character, Double, Boolean> value, long currentTimestamp) {
-			return (long) value.f0;
-		}
-
-		@Override
-		public long extractWatermark(Tuple5<Integer, String, Character, Double, Boolean> value,
-				long currentTimestamp) {
-			return (long) value.f0 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-
-	private static class MyFlatMapFunction implements FlatMapFunction<Tuple4<Integer, String, Double,
-			Boolean>, OuterPojo> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(Tuple4<Integer, String, Double, Boolean> value, Collector<OuterPojo> out) throws
-				Exception {
-			if (value.f3) {
-				for (int i = 0; i < 2; i++) {
-					out.collect(new OuterPojo(new InnerPojo((long) value.f0, value.f1), (long) i));
-				}
-			}
-		}
-	}
-
-	private class MyCoMapFunction implements CoMapFunction<OuterPojo, OuterPojo, String> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public String map1(OuterPojo value) {
-			return value.f0.f1;
-		}
-
-		@Override
-		public String map2(OuterPojo value) {
-			return value.f0.f1;
-		}
-	}
-
-	private class MyOutputSelector implements OutputSelector<Tuple2<Long, Tuple2<String, Long>>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Iterable<String> select(Tuple2<Long, Tuple2<String, Long>> value) {
-			List<String> output = new ArrayList<>();
-			if (value.f0 == 10) {
-				output.add("iterate");
-				output.add("firstOutput");
-			} else if (value.f0 == 20) {
-				output.add("secondOutput");
-			} else {
-				output.add("iterate");
-			}
-			return output;
-		}
-	}
-
-	private static class PrimeFilterFunction implements FilterFunction<Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public boolean filter(Long value) throws Exception {
-			if (value < 2) {
-				return false;
-			} else {
-				for (long i = 2; i < value; i++) {
-					if (value % i == 0) {
-						return false;
-					}
-				}
-			}
-			return true;
-		}
-	}
-
-	private static class DivisorsFlatMapFunction implements FlatMapFunction<Long, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(Long value, Collector<Long> out) throws Exception {
-			for (long i = 2; i <= value; i++) {
-				if (value % i == 0) {
-					out.collect(i);
-				}
-			}
-		}
-	}
-
-	private static class RectangleSource extends RichSourceFunction<Rectangle> {
-		private static final long serialVersionUID = 1L;
-		private transient Rectangle rectangle;
-
-		public void open(Configuration parameters) throws Exception {
-			rectangle = new Rectangle(100, 100);
-		}
-
-		@Override
-		public void run(SourceContext<Rectangle> ctx) throws Exception {
-			// emit once as the initializer of the delta trigger
-			ctx.collect(rectangle);
-			for (int i = 0; i < 100; i++) {
-				ctx.collect(rectangle);
-				rectangle = rectangle.next();
-			}
-		}
-
-		@Override
-		public void cancel() {
-		}
-	}
-
-	private static class RectangleMapFunction implements MapFunction<Rectangle, Tuple2<Rectangle, Integer>> {
-		private static final long serialVersionUID = 1L;
-		private int counter = 0;
-
-		@Override
-		public Tuple2<Rectangle, Integer> map(Rectangle value) throws Exception {
-			return new Tuple2<>(value, counter++);
-		}
-	}
-
-	private static class MyWindowMapFunction implements AllWindowFunction<Tuple2<Rectangle, Integer>, Tuple2<Rectangle, Integer>, GlobalWindow> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void apply(GlobalWindow window, Iterable<Tuple2<Rectangle, Integer>> values, Collector<Tuple2<Rectangle,
-				Integer>> out) throws Exception {
-			out.collect(values.iterator().next());
-		}
-	}
-
-	private static class SquareFlatMapFunction implements FlatMapFunction<Long, Long> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(Long value, Collector<Long> out) throws Exception {
-			for (long i = 0; i < value; i++) {
-				out.collect(value);
-			}
-		}
-	}
-
-	private static class Timestamp6 implements TimestampExtractor<Tuple2<Date, HashMap<Character, Integer>>> {
-
-		@Override
-		public long extractTimestamp(Tuple2<Date, HashMap<Character, Integer>> value,
-				long currentTimestamp) {
-			Calendar cal = Calendar.getInstance();
-			cal.setTime(value.f0);
-			return 12 * (cal.get(Calendar.YEAR)) + cal.get(Calendar.MONTH);
-		}
-
-		@Override
-		public long extractWatermark(Tuple2<Date, HashMap<Character, Integer>> value,
-				long currentTimestamp) {
-			Calendar cal = Calendar.getInstance();
-			cal.setTime(value.f0);
-			return 12 * (cal.get(Calendar.YEAR)) + cal.get(Calendar.MONTH) - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return 0;
-		}
-	}
-
-	private static class SalesReduceFunction implements ReduceFunction<Tuple2<Date, HashMap<Character, Integer>>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<Date, HashMap<Character, Integer>> reduce(Tuple2<Date, HashMap<Character, Integer>> value1,
-				Tuple2<Date,
-						HashMap<Character, Integer>> value2) throws Exception {
-			HashMap<Character, Integer> map1 = value1.f1;
-			HashMap<Character, Integer> map2 = value2.f1;
-			for (Character key : map2.keySet()) {
-				Integer volume1 = map1.get(key);
-				Integer volume2 = map2.get(key);
-				if (volume1 == null) {
-					volume1 = 0;
-				}
-				map1.put(key, volume1 + volume2);
-			}
-			return new Tuple2<>(value2.f0, map1);
-		}
-	}
-
-	private static class FlatMapFunction6 implements FlatMapFunction<Tuple2<Date, HashMap<Character, Integer>>, Tuple2<Integer,
-			Tuple2<Character, Integer>>> {
-
-		@Override
-		public void flatMap(Tuple2<Date, HashMap<Character, Integer>> value, Collector<Tuple2<Integer,
-				Tuple2<Character, Integer>>> out) throws Exception {
-			Calendar cal = Calendar.getInstance();
-			cal.setTime(value.f0);
-			for (Character key : value.f1.keySet()) {
-				out.collect(new Tuple2<>(cal.get(Calendar.MONTH)
-						+ 1,
-						new Tuple2<>(key, value.f1.get(key))));
-			}
-		}
-	}
-
-	private static class MapFunction6 implements MapFunction<Tuple2<Date, HashMap<Character, Integer>>, ArrayList<Character>> {
-
-		@Override
-		public ArrayList<Character> map(Tuple2<Date, HashMap<Character, Integer>> value)
-				throws Exception {
-			ArrayList<Character> list = new ArrayList<>();
-			for (Character ch : value.f1.keySet()) {
-				for (int i = 0; i < value.f1.get(ch); i++) {
-					list.add(ch);
-				}
-			}
-			Collections.sort(list);
-			return list;
-		}
-	}
-
-	// *************************************************************************
-	// DATA TYPES
-	// *************************************************************************
-
-	//Flink Pojo
-	public static class InnerPojo {
-		public Long f0;
-		public String f1;
-
-		//default constructor to qualify as Flink POJO
-		InnerPojo(){}
-
-		public InnerPojo(Long f0, String f1) {
-			this.f0 = f0;
-			this.f1 = f1;
-		}
-
-		@Override
-		public String toString() {
-			return "POJO(" + f0 + "," + f1 + ")";
-		}
-	}
-
-	// Nested class serialized with Kryo
-	public static class OuterPojo {
-		public InnerPojo f0;
-		public Long f1;
-
-		public OuterPojo(InnerPojo f0, Long f1) {
-			this.f0 = f0;
-			this.f1 = f1;
-		}
-
-		@Override
-		public String toString() {
-			return "POJO(" + f0 + "," + f1 + ")";
-		}
-	}
-
-	public static class Rectangle {
-
-		public int a;
-		public int b;
-
-		//default constructor to qualify as Flink POJO
-		public Rectangle() {}
-
-		public Rectangle(int a, int b) {
-			this.a = a;
-			this.b = b;
-		}
-
-		public Rectangle next() {
-			return new Rectangle(a + (b % 11), b + (a % 9));
-		}
-
-		@Override
-		public String toString() {
-			return "(" + a + "," + b + ")";
-		}
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/FromElementsFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/FromElementsFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/FromElementsFunctionTest.java
deleted file mode 100644
index 41bd381..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/FromElementsFunctionTest.java
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.api.java.typeutils.ValueTypeInfo;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.testutils.CommonTestUtils;
-import org.apache.flink.streaming.api.functions.source.FromElementsFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.types.Value;
-import org.apache.flink.util.ExceptionUtils;
-
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests for the {@link org.apache.flink.streaming.api.functions.source.FromElementsFunction}.
- */
-public class FromElementsFunctionTest {
-	
-	@Test
-	public void testStrings() {
-		try {
-			String[] data = { "Oh", "boy", "what", "a", "show", "!"};
-
-			FromElementsFunction<String> source = new FromElementsFunction<String>(
-					BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), data);
-			
-			List<String> result = new ArrayList<String>();
-			source.run(new ListSourceContext<String>(result));
-			
-			assertEquals(Arrays.asList(data), result);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testNonJavaSerializableType() {
-		try {
-			MyPojo[] data = { new MyPojo(1, 2), new MyPojo(3, 4), new MyPojo(5, 6) };
-
-			FromElementsFunction<MyPojo> source = new FromElementsFunction<MyPojo>(
-					TypeExtractor.getForClass(MyPojo.class).createSerializer(new ExecutionConfig()), data);
-
-			List<MyPojo> result = new ArrayList<MyPojo>();
-			source.run(new ListSourceContext<MyPojo>(result));
-
-			assertEquals(Arrays.asList(data), result);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSerializationError() {
-		try {
-			TypeInformation<SerializationErrorType> info = 
-					new ValueTypeInfo<SerializationErrorType>(SerializationErrorType.class);
-			
-			try {
-				new FromElementsFunction<SerializationErrorType>(
-					info.createSerializer(new ExecutionConfig()), new SerializationErrorType());
-				
-				fail("should fail with an exception");
-			}
-			catch (IOException e) {
-				assertTrue(ExceptionUtils.stringifyException(e).contains("test exception"));
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testDeSerializationError() {
-		try {
-			TypeInformation<DeserializeTooMuchType> info =
-					new ValueTypeInfo<DeserializeTooMuchType>(DeserializeTooMuchType.class);
-
-			FromElementsFunction<DeserializeTooMuchType> source = new FromElementsFunction<DeserializeTooMuchType>(
-					info.createSerializer(new ExecutionConfig()), new DeserializeTooMuchType());
-			
-			try {
-				source.run(new ListSourceContext<DeserializeTooMuchType>(new ArrayList<DeserializeTooMuchType>()));
-				fail("should fail with an exception");
-			}
-			catch (IOException e) {
-				assertTrue(ExceptionUtils.stringifyException(e).contains("user-defined serialization"));
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testCheckpointAndRestore() {
-		try {
-			final int NUM_ELEMENTS = 10000;
-			
-			List<Integer> data = new ArrayList<Integer>(NUM_ELEMENTS);
-			List<Integer> result = new ArrayList<Integer>(NUM_ELEMENTS);
-			
-			for (int i = 0; i < NUM_ELEMENTS; i++) {
-				data.add(i);
-			}
-			
-			final FromElementsFunction<Integer> source = new FromElementsFunction<Integer>(IntSerializer.INSTANCE, data);
-			final FromElementsFunction<Integer> sourceCopy = CommonTestUtils.createCopySerializable(source);
-			
-			final SourceFunction.SourceContext<Integer> ctx = new ListSourceContext<Integer>(result, 2L);
-			
-			final Throwable[] error = new Throwable[1];
-			
-			// run the source asynchronously
-			Thread runner = new Thread() {
-				@Override
-				public void run() {
-					try {
-						source.run(ctx);
-					}
-					catch (Throwable t) {
-						error[0] = t;
-					}
-				}
-			};
-			runner.start();
-			
-			// wait for a bit 
-			Thread.sleep(1000);
-			
-			// make a checkpoint
-			int count;
-			List<Integer> checkpointData = new ArrayList<Integer>(NUM_ELEMENTS);
-			
-			synchronized (ctx.getCheckpointLock()) {
-				count = source.snapshotState(566, System.currentTimeMillis());
-				checkpointData.addAll(result);
-			}
-			
-			// cancel the source
-			source.cancel();
-			runner.join();
-			
-			// check for errors
-			if (error[0] != null) {
-				System.err.println("Error in asynchronous source runner");
-				error[0].printStackTrace();
-				fail("Error in asynchronous source runner");
-			}
-			
-			// recovery run
-			SourceFunction.SourceContext<Integer> newCtx = new ListSourceContext<Integer>(checkpointData);
-			sourceCopy.restoreState(count);
-			
-			sourceCopy.run(newCtx);
-			
-			assertEquals(data, checkpointData);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	
-	// ------------------------------------------------------------------------
-	//  Test Types
-	// ------------------------------------------------------------------------
-	
-	public static class MyPojo {
-		
-		public long val1;
-		public int val2;
-
-		public MyPojo() {}
-		
-		public MyPojo(long val1, int val2) {
-			this.val1 = val1;
-			this.val2 = val2;
-		}
-
-		@Override
-		public int hashCode() {
-			return this.val2;
-		}
-
-		@Override
-		public boolean equals(Object obj) {
-			if (obj instanceof MyPojo) {
-				MyPojo that = (MyPojo) obj;
-				return this.val1 == that.val1 && this.val2 == that.val2; 
-			}
-			else {
-				return false;
-			}
-		}
-	}
-	
-	public static class SerializationErrorType implements Value {
-
-		private static final long serialVersionUID = -6037206294939421807L;
-
-		@Override
-		public void write(DataOutputView out) throws IOException {
-			throw new IOException("test exception");
-		}
-
-		@Override
-		public void read(DataInputView in) throws IOException {
-			throw new IOException("test exception");
-		}
-	}
-
-	public static class DeserializeTooMuchType implements Value {
-
-		private static final long serialVersionUID = -6037206294939421807L;
-
-		@Override
-		public void write(DataOutputView out) throws IOException {
-			out.writeInt(42);
-		}
-
-		@Override
-		public void read(DataInputView in) throws IOException {
-			in.readLong();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java
deleted file mode 100644
index e4dadf0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/ListSourceContext.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-import java.util.List;
-
-/**
- * Mock context that collects elements in a List.
- * 
- * @param <T> Type of the collected elements.
- */
-public class ListSourceContext<T> implements SourceFunction.SourceContext<T> {
-	
-	private final Object lock = new Object();
-	
-	private final List<T> target;
-
-	private final long delay;
-	
-	
-	public ListSourceContext(List<T> target) {
-		this(target, 0L);
-	}
-
-	public ListSourceContext(List<T> target, long delay) {
-		this.target = target;
-		this.delay = delay;
-	}
-
-	@Override
-	public void collect(T element) {
-		target.add(element);
-		
-		if (delay > 0) {
-			try {
-				Thread.sleep(delay);
-			}
-			catch (InterruptedException e) {
-				// ignore
-			}
-		}
-	}
-
-	@Override
-	public void collectWithTimestamp(T element, long timestamp) {
-		target.add(element);
-	}
-
-	@Override
-	public void emitWatermark(Watermark mark) {
-		// don't do anything
-	}
-
-	@Override
-	public Object getCheckpointLock() {
-		return lock;
-	}
-
-	@Override
-	public void close() {
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java
deleted file mode 100644
index f7c6e53..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-import java.io.ByteArrayOutputStream;
-import java.io.PrintStream;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests for the {@link org.apache.flink.streaming.api.functions.sink.PrintSinkFunction}.
- */
-public class PrintSinkFunctionTest {
-
-	public PrintStream printStreamOriginal = System.out;
-	private String line = System.lineSeparator();
-
-	@Test
-	public void testPrintSinkStdOut(){
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		PrintStream stream = new PrintStream(baos);
-		System.setOut(stream);
-
-		final StreamingRuntimeContext ctx = Mockito.mock(StreamingRuntimeContext.class);
-
-		PrintSinkFunction<String> printSink = new PrintSinkFunction<>();
-		printSink.setRuntimeContext(ctx);
-		try {
-			printSink.open(new Configuration());
-		} catch (Exception e) {
-			Assert.fail();
-		}
-		printSink.setTargetToStandardOut();
-		printSink.invoke("hello world!");
-
-		assertEquals("Print to System.out", printSink.toString());
-		assertEquals("hello world!" + line, baos.toString());
-
-		printSink.close();
-		stream.close();
-	}
-
-	@Test
-	public void testPrintSinkStdErr(){
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		PrintStream stream = new PrintStream(baos);
-		System.setOut(stream);
-
-		final StreamingRuntimeContext ctx = Mockito.mock(StreamingRuntimeContext.class);
-
-		PrintSinkFunction<String> printSink = new PrintSinkFunction<>();
-		printSink.setRuntimeContext(ctx);
-		try {
-			printSink.open(new Configuration());
-		} catch (Exception e) {
-			Assert.fail();
-		}
-		printSink.setTargetToStandardErr();
-		printSink.invoke("hello world!");
-
-		assertEquals("Print to System.err", printSink.toString());
-		assertEquals("hello world!" + line, baos.toString());
-
-		printSink.close();
-		stream.close();
-	}
-
-	@Test
-	public void testPrintSinkWithPrefix(){
-		ByteArrayOutputStream baos = new ByteArrayOutputStream();
-		PrintStream stream = new PrintStream(baos);
-		System.setOut(stream);
-
-		final StreamingRuntimeContext ctx = Mockito.mock(StreamingRuntimeContext.class);
-		Mockito.when(ctx.getNumberOfParallelSubtasks()).thenReturn(2);
-		Mockito.when(ctx.getIndexOfThisSubtask()).thenReturn(1);
-
-		PrintSinkFunction<String> printSink = new PrintSinkFunction<>();
-		printSink.setRuntimeContext(ctx);
-		try {
-			printSink.open(new Configuration());
-		} catch (Exception e) {
-			Assert.fail();
-		}
-		printSink.setTargetToStandardErr();
-		printSink.invoke("hello world!");
-
-		assertEquals("Print to System.err", printSink.toString());
-		assertEquals("2> hello world!" + line, baos.toString());
-
-		printSink.close();
-		stream.close();
-	}
-
-	@After
-	public void restoreSystemOut() {
-		System.setOut(printStreamOriginal);
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java
deleted file mode 100644
index 8f4acde..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.sink;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.ServerSocket;
-import java.net.Socket;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for the {@link org.apache.flink.streaming.api.functions.sink.SocketClientSink}.
- */
-@SuppressWarnings("serial")
-public class SocketClientSinkTest extends TestLogger {
-
-	private static final String TEST_MESSAGE = "testSocketSinkInvoke";
-
-	private static final String EXCEPTION_MESSGAE = "Failed to send message '" + TEST_MESSAGE + "\n'";
-
-	private static final String host = "127.0.0.1";
-
-	private SerializationSchema<String, byte[]> simpleSchema = new SerializationSchema<String, byte[]>() {
-		@Override
-		public byte[] serialize(String element) {
-			return element.getBytes();
-		}
-	};
-
-	@Test
-	public void testSocketSink() throws Exception {
-		final ServerSocket server = new ServerSocket(0);
-		final int port = server.getLocalPort();
-
-		final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
-
-		Thread sinkRunner = new Thread("Test sink runner") {
-			@Override
-			public void run() {
-				try {
-					SocketClientSink<String> simpleSink = new SocketClientSink<>(host, port, simpleSchema, 0);
-					simpleSink.open(new Configuration());
-					simpleSink.invoke(TEST_MESSAGE + '\n');
-					simpleSink.close();
-				}
-				catch (Throwable t) {
-					error.set(t);
-				}
-			}
-		};
-
-		sinkRunner.start();
-
-		Socket sk = server.accept();
-		BufferedReader rdr = new BufferedReader(new InputStreamReader(sk.getInputStream()));
-
-		String value = rdr.readLine();
-
-		sinkRunner.join();
-		server.close();
-
-		if (error.get() != null) {
-			Throwable t = error.get();
-			t.printStackTrace();
-			fail("Error in spawned thread: " + t.getMessage());
-		}
-
-		assertEquals(TEST_MESSAGE, value);
-	}
-
-	@Test
-	public void testSinkAutoFlush() throws Exception {
-		final ServerSocket server = new ServerSocket(0);
-		final int port = server.getLocalPort();
-
-		final SocketClientSink<String> simpleSink = new SocketClientSink<>(host, port, simpleSchema, 0, true);
-		simpleSink.open(new Configuration());
-
-		final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
-
-		Thread sinkRunner = new Thread("Test sink runner") {
-			@Override
-			public void run() {
-				try {
-					// need two messages here: send a fin to cancel the client state:FIN_WAIT_2 while the server is CLOSE_WAIT
-					simpleSink.invoke(TEST_MESSAGE + '\n');
-				}
-				catch (Throwable t) {
-					error.set(t);
-				}
-			}
-		};
-
-		sinkRunner.start();
-
-		Socket sk = server.accept();
-		BufferedReader rdr = new BufferedReader(new InputStreamReader(sk.getInputStream()));
-		String value = rdr.readLine();
-
-		sinkRunner.join();
-		simpleSink.close();
-		server.close();
-
-		if (error.get() != null) {
-			Throwable t = error.get();
-			t.printStackTrace();
-			fail("Error in spawned thread: " + t.getMessage());
-		}
-
-		assertEquals(TEST_MESSAGE, value);
-	}
-
-	@Test
-	public void testSocketSinkNoRetry() throws Exception {
-		final ServerSocket server = new ServerSocket(0);
-		final int port = server.getLocalPort();
-
-		try {
-			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
-
-			Thread serverRunner = new Thread("Test server runner") {
-
-				@Override
-				public void run() {
-					try {
-						Socket sk = server.accept();
-						sk.close();
-					}
-					catch (Throwable t) {
-						error.set(t);
-					}
-				}
-			};
-			serverRunner.start();
-
-			SocketClientSink<String> simpleSink = new SocketClientSink<>(host, port, simpleSchema, 0, true);
-			simpleSink.open(new Configuration());
-
-			// wait socket server to close
-			serverRunner.join();
-			if (error.get() != null) {
-				Throwable t = error.get();
-				t.printStackTrace();
-				fail("Error in server thread: " + t.getMessage());
-			}
-
-			try {
-				// socket should be closed, so this should trigger a re-try
-				// need two messages here: send a fin to cancel the client state:FIN_WAIT_2 while the server is CLOSE_WAIT
-				while (true) { // we have to do this more often as the server side closed is not guaranteed to be noticed immediately
-					simpleSink.invoke(TEST_MESSAGE + '\n');
-				}
-			}
-			catch (IOException e) {
-				// check whether throw a exception that reconnect failed.
-				assertTrue("Wrong exception", e.getMessage().contains(EXCEPTION_MESSGAE));
-			}
-			catch (Exception e) {
-				fail("wrong exception: " + e.getClass().getName() + " - " + e.getMessage());
-			}
-
-			assertEquals(0, simpleSink.getCurrentNumberOfRetries());
-		}
-		finally {
-			IOUtils.closeQuietly(server);
-		}
-	}
-
-	@Test
-	public void testRetry() throws Exception {
-
-		final ServerSocket[] serverSocket = new ServerSocket[1];
-		final ExecutorService[] executor = new ExecutorService[1];
-
-		try {
-			serverSocket[0] = new ServerSocket(0);
-			executor[0] = Executors.newCachedThreadPool();
-
-			int port = serverSocket[0].getLocalPort();
-
-			Callable<Void> serverTask = new Callable<Void>() {
-				@Override
-				public Void call() throws Exception {
-					Socket socket = serverSocket[0].accept();
-
-					BufferedReader reader = new BufferedReader(new InputStreamReader(
-							socket.getInputStream()));
-
-					String value = reader.readLine();
-					assertEquals("0", value);
-
-					socket.close();
-					return null;
-				}
-			};
-
-			Future<Void> serverFuture = executor[0].submit(serverTask);
-
-			final SocketClientSink<String> sink = new SocketClientSink<>(
-					host, serverSocket[0].getLocalPort(), simpleSchema, -1, true);
-
-			// Create the connection
-			sink.open(new Configuration());
-
-			// Initial payload => this will be received by the server an then the socket will be
-			// closed.
-			sink.invoke("0\n");
-
-			// Get future an make sure there was no problem. This will rethrow any Exceptions from
-			// the server.
-			serverFuture.get();
-
-			// Shutdown the server socket
-			serverSocket[0].close();
-			assertTrue(serverSocket[0].isClosed());
-
-			// No retries expected at this point
-			assertEquals(0, sink.getCurrentNumberOfRetries());
-
-			final CountDownLatch retryLatch = new CountDownLatch(1);
-			final CountDownLatch again = new CountDownLatch(1);
-
-			Callable<Void> sinkTask = new Callable<Void>() {
-				@Override
-				public Void call() throws Exception {
-					// Send next payload => server is down, should try to reconnect.
-
-					// We need to send more than just one packet to notice the closed connection.
-					while (retryLatch.getCount() != 0) {
-						sink.invoke("1\n");
-					}
-
-					return null;
-				}
-			};
-
-			Future<Void> sinkFuture = executor[0].submit(sinkTask);
-
-			while (sink.getCurrentNumberOfRetries() == 0) {
-				// Wait for a retry
-				Thread.sleep(100);
-			}
-
-			// OK the poor guy retried to write
-			retryLatch.countDown();
-
-			// Restart the server
-			serverSocket[0] = new ServerSocket(port);
-			Socket socket = serverSocket[0].accept();
-
-			BufferedReader reader = new BufferedReader(new InputStreamReader(
-					socket.getInputStream()));
-
-			// Wait for the reconnect
-			String value = reader.readLine();
-
-			assertEquals("1", value);
-
-			// OK the sink re-connected. :)
-		}
-		finally {
-			if (serverSocket[0] != null) {
-				serverSocket[0].close();
-			}
-
-			if (executor[0] != null) {
-				executor[0].shutdown();
-			}
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunctionTest.java
deleted file mode 100644
index 2d9921a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/FileMonitoringFunctionTest.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *	 http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.junit.Test;
-
-/**
- * Tests for the {@link org.apache.flink.streaming.api.functions.source.FileMonitoringFunction}.
- */
-public class FileMonitoringFunctionTest {
-
-	@Test
-	public void testForEmptyLocation() throws Exception {
-		final FileMonitoringFunction fileMonitoringFunction
-			= new FileMonitoringFunction("?non-existing-path", 1L, FileMonitoringFunction.WatchType.ONLY_NEW_FILES);
-
-        new Thread() {
-            @Override
-            public void run() {
-                try {
-                    Thread.sleep(1000L);
-                } catch (InterruptedException e) {
-                    e.printStackTrace();
-                }
-                fileMonitoringFunction.cancel();
-            }
-        }.start();
-
-		fileMonitoringFunction.run(
-            new StreamSource.NonWatermarkContext<Tuple3<String, Long, Long>>(
-                new Object(),
-                new Output<StreamRecord<Tuple3<String, Long, Long>>>() {
-                    @Override
-                    public void emitWatermark(Watermark mark) { }
-                    @Override
-                    public void collect(StreamRecord<Tuple3<String, Long, Long>> record) { }
-                    @Override
-                    public void close() { }
-                })
-        );
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunctionTest.java
deleted file mode 100644
index 3398451..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/source/SocketTextStreamFunctionTest.java
+++ /dev/null
@@ -1,349 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.functions.source;
-
-import org.apache.commons.io.IOUtils;
-
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-import org.junit.Test;
-
-import java.io.EOFException;
-import java.io.OutputStreamWriter;
-import java.net.ServerSocket;
-import java.net.Socket;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests for the {@link org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction}.
- */
-public class SocketTextStreamFunctionTest {
-
-	private static final String LOCALHOST = "127.0.0.1";
-
-
-	@Test
-	public void testSocketSourceSimpleOutput() throws Exception {
-		ServerSocket server = new ServerSocket(0);
-		Socket channel = null;
-		
-		try {
-			SocketTextStreamFunction source = new SocketTextStreamFunction(LOCALHOST, server.getLocalPort(), '\n', 0);
-	
-			SocketSourceThread runner = new SocketSourceThread(source, "test1", "check");
-			runner.start();
-	
-			channel = server.accept();
-			OutputStreamWriter writer = new OutputStreamWriter(channel.getOutputStream());
-			
-			writer.write("test1\n");
-			writer.write("check\n");
-			writer.flush();
-			runner.waitForNumElements(2);
-
-			runner.cancel();
-			runner.interrupt();
-			
-			runner.waitUntilDone();
-			
-			channel.close();
-		}
-		finally {
-			if (channel != null) {
-				IOUtils.closeQuietly(channel);
-			}
-			IOUtils.closeQuietly(server);
-		}
-	}
-
-	@Test
-	public void testExitNoRetries() throws Exception {
-		ServerSocket server = new ServerSocket(0);
-		Socket channel = null;
-
-		try {
-			SocketTextStreamFunction source = new SocketTextStreamFunction(LOCALHOST, server.getLocalPort(), '\n', 0);
-
-			SocketSourceThread runner = new SocketSourceThread(source);
-			runner.start();
-
-			channel = server.accept();
-			channel.close();
-			
-			try {
-				runner.waitUntilDone();
-			}
-			catch (Exception e) {
-				assertTrue(e.getCause() instanceof EOFException);
-			}
-		}
-		finally {
-			if (channel != null) {
-				IOUtils.closeQuietly(channel);
-			}
-			IOUtils.closeQuietly(server);
-		}
-	}
-
-	@Test
-	public void testSocketSourceOutputWithRetries() throws Exception {
-		ServerSocket server = new ServerSocket(0);
-		Socket channel = null;
-
-		try {
-			SocketTextStreamFunction source = new SocketTextStreamFunction(LOCALHOST, server.getLocalPort(), '\n', 10, 100);
-
-			SocketSourceThread runner = new SocketSourceThread(source, "test1", "check");
-			runner.start();
-
-			// first connection: nothing
-			channel = server.accept();
-			channel.close();
-
-			// second connection: first string
-			channel = server.accept();
-			OutputStreamWriter writer = new OutputStreamWriter(channel.getOutputStream());
-			writer.write("test1\n");
-			writer.close();
-			channel.close();
-
-			// third connection: nothing
-			channel = server.accept();
-			channel.close();
-
-			// forth connection: second string
-			channel = server.accept();
-			writer = new OutputStreamWriter(channel.getOutputStream());
-			writer.write("check\n");
-			writer.flush();
-
-			runner.waitForNumElements(2);
-			runner.cancel();
-			runner.waitUntilDone();
-		}
-		finally {
-			if (channel != null) {
-				IOUtils.closeQuietly(channel);
-			}
-			IOUtils.closeQuietly(server);
-		}
-	}
-
-	@Test
-	public void testSocketSourceOutputInfiniteRetries() throws Exception {
-		ServerSocket server = new ServerSocket(0);
-		Socket channel = null;
-
-		try {
-			SocketTextStreamFunction source = new SocketTextStreamFunction(LOCALHOST, server.getLocalPort(), '\n', -1, 100);
-
-			SocketSourceThread runner = new SocketSourceThread(source, "test1", "check");
-			runner.start();
-
-			// first connection: nothing
-			channel = server.accept();
-			channel.close();
-
-			// second connection: first string
-			channel = server.accept();
-			OutputStreamWriter writer = new OutputStreamWriter(channel.getOutputStream());
-			writer.write("test1\n");
-			writer.close();
-			channel.close();
-
-			// third connection: nothing
-			channel = server.accept();
-			channel.close();
-
-			// forth connection: second string
-			channel = server.accept();
-			writer = new OutputStreamWriter(channel.getOutputStream());
-			writer.write("check\n");
-			writer.flush();
-
-			runner.waitForNumElements(2);
-			runner.cancel();
-			runner.waitUntilDone();
-		}
-		finally {
-			if (channel != null) {
-				IOUtils.closeQuietly(channel);
-			}
-			IOUtils.closeQuietly(server);
-		}
-	}
-
-	@Test
-	public void testSocketSourceOutputAcrossRetries() throws Exception {
-		ServerSocket server = new ServerSocket(0);
-		Socket channel = null;
-
-		try {
-			SocketTextStreamFunction source = new SocketTextStreamFunction(LOCALHOST, server.getLocalPort(), '\n', 10, 100);
-
-			SocketSourceThread runner = new SocketSourceThread(source, "test1", "check1", "check2");
-			runner.start();
-
-			// first connection: nothing
-			channel = server.accept();
-			channel.close();
-
-			// second connection: first string
-			channel = server.accept();
-			OutputStreamWriter writer = new OutputStreamWriter(channel.getOutputStream());
-			writer.write("te");
-			writer.close();
-			channel.close();
-
-			// third connection: nothing
-			channel = server.accept();
-			channel.close();
-
-			// forth connection: second string
-			channel = server.accept();
-			writer = new OutputStreamWriter(channel.getOutputStream());
-			writer.write("st1\n");
-			writer.write("check1\n");
-			writer.write("check2\n");
-			writer.flush();
-
-			runner.waitForNumElements(2);
-			runner.cancel();
-			runner.waitUntilDone();
-		}
-		finally {
-			if (channel != null) {
-				IOUtils.closeQuietly(channel);
-			}
-			IOUtils.closeQuietly(server);
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-
-	private static class SocketSourceThread extends Thread {
-		
-		private final Object sync = new Object();
-		
-		private final SocketTextStreamFunction socketSource;
-		
-		private final String[] expectedData;
-		
-		private volatile Throwable error;
-		private volatile int numElementsReceived;
-		private volatile boolean canceled;
-		private volatile boolean done;
-		
-		public SocketSourceThread(SocketTextStreamFunction socketSource, String... expectedData) {
-			this.socketSource = socketSource;
-			this.expectedData = expectedData;
-		}
-
-		public void run() {
-			try {
-				SourceFunction.SourceContext<String> ctx = new SourceFunction.SourceContext<String>() {
-					
-					private final Object lock = new Object();
-					
-					@Override
-					public void collect(String element) {
-						int pos = numElementsReceived;
-						
-						// make sure waiter know of us
-						synchronized (sync) {
-							numElementsReceived++;
-							sync.notifyAll();
-						}
-						
-						if (expectedData != null && expectedData.length > pos) {
-							assertEquals(expectedData[pos], element);
-						}
-					}
-
-					@Override
-					public void collectWithTimestamp(String element, long timestamp) {
-						collect(element);
-					}
-
-					@Override
-					public void emitWatermark(Watermark mark) {}
-
-					@Override
-					public Object getCheckpointLock() {
-						return lock;
-					}
-
-					@Override
-					public void close() {}
-				};
-				
-				socketSource.run(ctx);
-			}
-			catch (Throwable t) {
-				synchronized (sync) {
-					if (!canceled) {
-						error = t;
-					}
-					sync.notifyAll();
-				}
-			}
-			finally {
-				synchronized (sync) {
-					done = true;
-					sync.notifyAll();
-				}
-			}
-		}
-		
-		public void cancel() {
-			synchronized (sync) {
-				canceled = true;
-				socketSource.cancel();
-				interrupt();
-			}
-		}
-
-		public void waitForNumElements(int numElements) throws InterruptedException {
-			synchronized (sync) {
-				while (error == null && !canceled && !done && numElementsReceived < numElements) {
-					sync.wait();
-				}
-
-				if (error != null) {
-					throw new RuntimeException("Error in source thread", error);
-				}
-				if (canceled) {
-					throw new RuntimeException("canceled");
-				}
-				if (done) {
-					throw new RuntimeException("Exited cleanly before expected number of elements");
-				}
-			}
-		}
-
-		public void waitUntilDone() throws InterruptedException {
-			join();
-
-			if (error != null) {
-				throw new RuntimeException("Error in source thread", error);
-			}
-		}
-	}
-}
\ No newline at end of file

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


[30/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
deleted file mode 100644
index 80df72e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.streamrecord;
-
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-/**
- * An element in a data stream. Can be a record or a Watermark.
- */
-public abstract class StreamElement {
-	
-	/**
-	 * Checks whether this element is a watermark.
-	 * @return True, if this element is a watermark, false otherwise.
-	 */
-	public final boolean isWatermark() {
-		return getClass() == Watermark.class;
-	}
-
-	/**
-	 * Checks whether this element is a record.
-	 * @return True, if this element is a record, false otherwise.
-	 */
-	public final boolean isRecord() {
-		return getClass() == StreamRecord.class;
-	}
-
-	/**
-	 * Casts this element into a StreamRecord.
-	 * @return This element as a stream record.
-	 * @throws java.lang.ClassCastException Thrown, if this element is actually not a stream record.
-	 */
-	@SuppressWarnings("unchecked")
-	public final <E> StreamRecord<E> asRecord() {
-		return (StreamRecord<E>) this;
-	}
-
-	/**
-	 * Casts this element into a Watermark.
-	 * @return This element as a Watermark.
-	 * @throws java.lang.ClassCastException Thrown, if this element is actually not a Watermark.
-	 */
-	public final Watermark asWatermark() {
-		return (Watermark) this;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
deleted file mode 100644
index 348b974..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.streamrecord;
-
-/**
- * One value in a data stream. This stores the value and the associated timestamp.
- * 
- * @param <T> The type encapsulated with the stream record.
- */
-public class StreamRecord<T> extends StreamElement {
-	
-	/** The actual value held by this record */
-	private T value;
-	
-	/** The timestamp of the record */
-	private long timestamp;
-
-	/**
-	 * Creates a new {@link StreamRecord} wrapping the given value. The timestamp is set to the
-	 * result of {@code new Instant(0)}.
-	 */
-	public StreamRecord(T value) {
-		this(value, Long.MIN_VALUE + 1);
-		// be careful to set it to MIN_VALUE + 1, because MIN_VALUE is reserved as the
-		// special tag to signify that a transmitted element is a Watermark in StreamRecordSerializer
-	}
-
-	/**
-	 * Creates a new {@link StreamRecord} wrapping the given value. The timestamp is set to the
-	 * given timestamp.
-	 *
-	 * @param value The value to wrap in this {@link StreamRecord}
-	 * @param timestamp The timestamp in milliseconds
-	 */
-	public StreamRecord(T value, long timestamp) {
-		this.value = value;
-		this.timestamp = timestamp;
-	}
-
-	/**
-	 * Returns the value wrapped in this stream value.
-	 */
-	public T getValue() {
-		return value;
-	}
-
-	/**
-	 * Returns the timestamp associated with this stream value in milliseconds.
-	 */
-	public long getTimestamp() {
-		return timestamp;
-	}
-
-	/**
-	 * Replace the currently stored value by the given new value. This returns a StreamElement
-	 * with the generic type parameter that matches the new value while keeping the old
-	 * timestamp.
-	 *
-	 * @param element Element to set in this stream value
-	 * @return Returns the StreamElement with replaced value
-	 */
-	@SuppressWarnings("unchecked")
-	public <X> StreamRecord<X> replace(X element) {
-		this.value = (T) element;
-		return (StreamRecord<X>) this;
-	}
-
-	/**
-	 * Replace the currently stored value by the given new value and the currently stored
-	 * timestamp with the new timestamp. This returns a StreamElement with the generic type
-	 * parameter that matches the new value.
-	 *
-	 * @param value The new value to wrap in this {@link StreamRecord}
-	 * @param timestamp The new timestamp in milliseconds
-	 * @return Returns the StreamElement with replaced value
-	 */
-	@SuppressWarnings("unchecked")
-	public <X> StreamRecord<X> replace(X value, long timestamp) {
-		this.timestamp = timestamp;
-		this.value = (T) value;
-		return (StreamRecord<X>) this;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		StreamRecord<?> that = (StreamRecord<?>) o;
-
-		return value.equals(that.value) && timestamp == that.timestamp;
-	}
-
-	@Override
-	public int hashCode() {
-		int result = value != null ? value.hashCode() : 0;
-		result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
-		return result;
-	}
-
-	@Override
-	public String toString() {
-		return "Record{" + value + "; " + timestamp + '}';
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java
deleted file mode 100644
index d47da50..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUStreamRecord<?>WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.streamrecord;
-
-import java.io.IOException;
-
-import com.google.common.base.Preconditions;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-/**
- * Serializer for {@link StreamRecord}. This version ignores timestamps and only deals with
- * the element.
- *
- * <p>
- * {@link MultiplexingStreamRecordSerializer} is a version that deals with timestamps and also
- * multiplexes {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks} in the same
- * stream with {@link StreamRecord StreamRecords}.
- *
- * @see MultiplexingStreamRecordSerializer
- *
- * @param <T> The type of value in the {@link StreamRecord}
- */
-public final class StreamRecordSerializer<T> extends TypeSerializer<StreamRecord<T>> {
-
-	private static final long serialVersionUID = 1L;
-
-	private final TypeSerializer<T> typeSerializer;
-	
-
-	public StreamRecordSerializer(TypeSerializer<T> serializer) {
-		if (serializer instanceof StreamRecordSerializer) {
-			throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer);
-		}
-		this.typeSerializer = Preconditions.checkNotNull(serializer);
-	}
-
-	public TypeSerializer<T> getContainedTypeSerializer() {
-		return this.typeSerializer;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  General serializer and type utils
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamRecordSerializer<T> duplicate() {
-		TypeSerializer<T> serializerCopy = typeSerializer.duplicate();
-		return serializerCopy == typeSerializer ? this : new StreamRecordSerializer<T>(serializerCopy);
-	}
-
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public int getLength() {
-		return typeSerializer.getLength();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Type serialization, copying, instantiation
-	// ------------------------------------------------------------------------
-
-	@Override
-	public StreamRecord<T> createInstance() {
-		try {
-			return new StreamRecord<T>(typeSerializer.createInstance());
-		} catch (Exception e) {
-			throw new RuntimeException("Cannot instantiate StreamRecord.", e);
-		}
-	}
-	
-	@Override
-	public StreamRecord<T> copy(StreamRecord<T> from) {
-		return new StreamRecord<T>(typeSerializer.copy(from.getValue()), from.getTimestamp());
-	}
-
-	@Override
-	public StreamRecord<T> copy(StreamRecord<T> from, StreamRecord<T> reuse) {
-		reuse.replace(typeSerializer.copy(from.getValue(), reuse.getValue()), 0);
-		return reuse;
-	}
-
-	@Override
-	public void serialize(StreamRecord<T> value, DataOutputView target) throws IOException {
-		typeSerializer.serialize(value.getValue(), target);
-	}
-	
-	@Override
-	public StreamRecord<T> deserialize(DataInputView source) throws IOException {
-		T element = typeSerializer.deserialize(source);
-		return new StreamRecord<T>(element, 0);
-	}
-
-	@Override
-	public StreamRecord<T> deserialize(StreamRecord<T> reuse, DataInputView source) throws IOException {
-		T element = typeSerializer.deserialize(reuse.getValue(), source);
-		reuse.replace(element, 0);
-		return reuse;
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		typeSerializer.copy(source, target);
-	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof StreamRecordSerializer) {
-			StreamRecordSerializer<?> other = (StreamRecordSerializer<?>) obj;
-
-			return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof StreamRecordSerializer;
-	}
-
-	@Override
-	public int hashCode() {
-		return typeSerializer.hashCode();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java
deleted file mode 100644
index ec90bff..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.tasks;
-
-import static java.util.Objects.requireNonNull;
-
-/**
- * A special exception that signifies that the cause exception came from a chained operator.
- */
-public class ExceptionInChainedOperatorException extends RuntimeException {
-
-	private static final long serialVersionUID = 1L;
-
-	public ExceptionInChainedOperatorException(Throwable cause) {
-		this("Could not forward element to next operator", cause);
-	}
-
-	public ExceptionInChainedOperatorException(String message, Throwable cause) {
-		super(message, requireNonNull(cause));
-	}
-	
-	public Throwable getOriginalCause() {
-		Throwable ex = this;
-		do {
-			ex = ex.getCause();
-		} while (ex instanceof ExceptionInChainedOperatorException);
-		return ex; 
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
deleted file mode 100644
index 5316ae4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.tasks;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.runtime.io.StreamInputProcessor;
-
-public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamOperator<IN, OUT>> {
-
-	private StreamInputProcessor<IN> inputProcessor;
-	
-	private volatile boolean running = true;
-
-	@Override
-	public void init() throws Exception {
-		StreamConfig configuration = getConfiguration();
-		
-		TypeSerializer<IN> inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader());
-		int numberOfInputs = configuration.getNumberOfInputs();
-
-		if (numberOfInputs > 0) {
-			InputGate[] inputGates = getEnvironment().getAllInputGates();
-			inputProcessor = new StreamInputProcessor<IN>(inputGates, inSerializer,
-					getCheckpointBarrierListener(), 
-					configuration.getCheckpointMode(),
-					getEnvironment().getIOManager(),
-					getExecutionConfig().areTimestampsEnabled());
-
-			// make sure that stream tasks report their I/O statistics
-			AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry();
-			AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter();
-			inputProcessor.setReporter(reporter);
-		}
-	}
-
-	@Override
-	protected void run() throws Exception {
-		// cache some references on the stack, to make the code more JIT friendly
-		final OneInputStreamOperator<IN, OUT> operator = this.headOperator;
-		final StreamInputProcessor<IN> inputProcessor = this.inputProcessor;
-		final Object lock = getCheckpointLock();
-		
-		while (running && inputProcessor.processInput(operator, lock)) {
-			checkTimerException();
-		}
-	}
-
-	@Override
-	protected void cleanup() throws Exception {
-		inputProcessor.cleanup();
-	}
-
-	@Override
-	protected void cancelTask() {
-		running = false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
deleted file mode 100644
index b42b888..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
+++ /dev/null
@@ -1,326 +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.tasks;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.io.CollectorWrapper;
-import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
-import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.runtime.io.StreamRecordWriter;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class OperatorChain<OUT> {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(OperatorChain.class);
-	
-	private final StreamOperator<?>[] allOperators;
-	
-	private final RecordWriterOutput<?>[] streamOutputs;
-	
-	private final Output<StreamRecord<OUT>> chainEntryPoint;
-	
-
-	public OperatorChain(StreamTask<OUT, ?> containingTask,
-							StreamOperator<OUT> headOperator,
-							AccumulatorRegistry.Reporter reporter) {
-		
-		final ClassLoader userCodeClassloader = containingTask.getUserCodeClassLoader();
-		final StreamConfig configuration = containingTask.getConfiguration();
-		final boolean enableTimestamps = containingTask.getExecutionConfig().areTimestampsEnabled();
-
-		// we read the chained configs, and the order of record writer registrations by output name
-		Map<Integer, StreamConfig> chainedConfigs = configuration.getTransitiveChainedTaskConfigs(userCodeClassloader);
-		chainedConfigs.put(configuration.getVertexID(), configuration);
-
-		// create the final output stream writers
-		// we iterate through all the out edges from this job vertex and create a stream output
-		List<StreamEdge> outEdgesInOrder = configuration.getOutEdgesInOrder(userCodeClassloader);
-		Map<StreamEdge, RecordWriterOutput<?>> streamOutputMap = new HashMap<>(outEdgesInOrder.size());
-		this.streamOutputs = new RecordWriterOutput<?>[outEdgesInOrder.size()];
-		
-		// from here on, we need to make sure that the output writers are shut down again on failure
-		boolean success = false;
-		try {
-			for (int i = 0; i < outEdgesInOrder.size(); i++) {
-				StreamEdge outEdge = outEdgesInOrder.get(i);
-				
-				RecordWriterOutput<?> streamOutput = createStreamOutput(
-						outEdge, chainedConfigs.get(outEdge.getSourceId()), i,
-						containingTask.getEnvironment(), enableTimestamps, reporter, containingTask.getName());
-	
-				this.streamOutputs[i] = streamOutput;
-				streamOutputMap.put(outEdge, streamOutput);
-			}
-	
-			// we create the chain of operators and grab the collector that leads into the chain
-			List<StreamOperator<?>> allOps = new ArrayList<>(chainedConfigs.size());
-			this.chainEntryPoint = createOutputCollector(containingTask, configuration,
-					chainedConfigs, userCodeClassloader, streamOutputMap, allOps);
-			
-			this.allOperators = allOps.toArray(new StreamOperator<?>[allOps.size() + 1]);
-			
-			// add the head operator to the end of the list
-			this.allOperators[this.allOperators.length - 1] = headOperator;
-			
-			success = true;
-		}
-		finally {
-			// make sure we clean up after ourselves in case of a failure after acquiring
-			// the first resources
-			if (!success) {
-				for (RecordWriterOutput<?> output : this.streamOutputs) {
-					if (output != null) {
-						output.close();
-						output.clearBuffers();
-					}
-				}
-			}
-		}
-		
-	}
-	
-	
-	public void broadcastCheckpointBarrier(long id, long timestamp) throws IOException, InterruptedException {
-		CheckpointBarrier barrier = new CheckpointBarrier(id, timestamp);
-		for (RecordWriterOutput<?> streamOutput : streamOutputs) {
-			streamOutput.broadcastEvent(barrier);
-		}
-	}
-	
-	public RecordWriterOutput<?>[] getStreamOutputs() {
-		return streamOutputs;
-	}
-	
-	public StreamOperator<?>[] getAllOperators() {
-		return allOperators;
-	}
-
-	public Output<StreamRecord<OUT>> getChainEntryPoint() {
-		return chainEntryPoint;
-	}
-
-	/**
-	 *
-	 * This method should be called before finishing the record emission, to make sure any data
-	 * that is still buffered will be sent. It also ensures that all data sending related
-	 * exceptions are recognized.
-	 *
-	 * @throws IOException Thrown, if the buffered data cannot be pushed into the output streams.
-	 */
-	public void flushOutputs() throws IOException {
-		for (RecordWriterOutput<?> streamOutput : getStreamOutputs()) {
-			streamOutput.flush();
-		}
-	}
-
-	/**
-	 * This method releases all resources of the record writer output. It stops the output
-	 * flushing thread (if there is one) and releases all buffers currently held by the output
-	 * serializers.
-	 *
-	 * <p>This method should never fail.
-	 */
-	public void releaseOutputs() {
-		try {
-			for (RecordWriterOutput<?> streamOutput : streamOutputs) {
-				streamOutput.close();
-			}
-		}
-		finally {
-			// make sure that we release the buffers in any case
-			for (RecordWriterOutput<?> output : streamOutputs) {
-				output.clearBuffers();
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  initialization utilities
-	// ------------------------------------------------------------------------
-	
-	private static <T> Output<StreamRecord<T>> createOutputCollector(
-			StreamTask<?, ?> containingTask,
-			StreamConfig operatorConfig,
-			Map<Integer, StreamConfig> chainedConfigs,
-			ClassLoader userCodeClassloader,
-			Map<StreamEdge, RecordWriterOutput<?>> streamOutputs,
-			List<StreamOperator<?>> allOperators)
-	{
-		// We create a wrapper that will encapsulate the chained operators and network outputs
-		OutputSelectorWrapper<T> outputSelectorWrapper = operatorConfig.getOutputSelectorWrapper(userCodeClassloader);
-		CollectorWrapper<T> wrapper = new CollectorWrapper<T>(outputSelectorWrapper);
-
-		// create collectors for the network outputs
-		for (StreamEdge outputEdge : operatorConfig.getNonChainedOutputs(userCodeClassloader)) {
-			@SuppressWarnings("unchecked")
-			RecordWriterOutput<T> output = (RecordWriterOutput<T>) streamOutputs.get(outputEdge);
-			wrapper.addCollector(output, outputEdge);
-		}
-
-		// Create collectors for the chained outputs
-		for (StreamEdge outputEdge : operatorConfig.getChainedOutputs(userCodeClassloader)) {
-			int outputId = outputEdge.getTargetId();
-			StreamConfig chainedOpConfig = chainedConfigs.get(outputId);
-
-			Output<StreamRecord<T>> output = createChainedOperator(
-					containingTask, chainedOpConfig, chainedConfigs, userCodeClassloader, streamOutputs, allOperators);
-			wrapper.addCollector(output, outputEdge);
-		}
-		return wrapper;
-	}
-	
-	private static <IN, OUT> Output<StreamRecord<IN>> createChainedOperator(
-			StreamTask<?, ?> containingTask,
-			StreamConfig operatorConfig,
-			Map<Integer, StreamConfig> chainedConfigs,
-			ClassLoader userCodeClassloader,
-			Map<StreamEdge, RecordWriterOutput<?>> streamOutputs,
-			List<StreamOperator<?>> allOperators)
-	{
-		// create the output that the operator writes to first. this may recursively create more operators
-		Output<StreamRecord<OUT>> output = createOutputCollector(
-				containingTask, operatorConfig, chainedConfigs, userCodeClassloader, streamOutputs, allOperators);
-
-		// now create the operator and give it the output collector to write its output to
-		OneInputStreamOperator<IN, OUT> chainedOperator = operatorConfig.getStreamOperator(userCodeClassloader);
-		chainedOperator.setup(containingTask, operatorConfig, output);
-
-		allOperators.add(chainedOperator);
-
-		if (containingTask.getExecutionConfig().isObjectReuseEnabled() || chainedOperator.isInputCopyingDisabled()) {
-			return new ChainingOutput<IN>(chainedOperator);
-		}
-		else {
-			TypeSerializer<IN> inSerializer = operatorConfig.getTypeSerializerIn1(userCodeClassloader);
-			return new CopyingChainingOutput<IN>(chainedOperator, inSerializer);
-		}
-	}
-	
-	private static <T> RecordWriterOutput<T> createStreamOutput(
-			StreamEdge edge, StreamConfig upStreamConfig, int outputIndex,
-			Environment taskEnvironment, boolean withTimestamps,
-			AccumulatorRegistry.Reporter reporter, String taskName)
-	{
-		TypeSerializer<T> outSerializer = upStreamConfig.getTypeSerializerOut(taskEnvironment.getUserClassLoader());
-
-		@SuppressWarnings("unchecked")
-		StreamPartitioner<T> outputPartitioner = (StreamPartitioner<T>) edge.getPartitioner();
-
-		LOG.debug("Using partitioner {} for output {} of task ", outputPartitioner, outputIndex, taskName);
-		
-		ResultPartitionWriter bufferWriter = taskEnvironment.getWriter(outputIndex);
-
-		StreamRecordWriter<SerializationDelegate<StreamRecord<T>>> output = 
-				new StreamRecordWriter<>(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout());
-		output.setReporter(reporter);
-		
-		return new RecordWriterOutput<T>(output, outSerializer, withTimestamps);
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Collectors for output chaining
-	// ------------------------------------------------------------------------ 
-
-	private static class ChainingOutput<T> implements Output<StreamRecord<T>> {
-		
-		protected final OneInputStreamOperator<T, ?> operator;
-
-		public ChainingOutput(OneInputStreamOperator<T, ?> operator) {
-			this.operator = operator;
-		}
-
-		@Override
-		public void collect(StreamRecord<T> record) {
-			try {
-				operator.setKeyContextElement(record);
-				operator.processElement(record);
-			}
-			catch (Exception e) {
-				throw new ExceptionInChainedOperatorException(e);
-			}
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			try {
-				operator.processWatermark(mark);
-			}
-			catch (Exception e) {
-				throw new ExceptionInChainedOperatorException(e);
-			}
-		}
-
-		@Override
-		public void close() {
-			try {
-				operator.close();
-			}
-			catch (Exception e) {
-				throw new ExceptionInChainedOperatorException(e);
-			}
-		}
-	}
-
-	private static class CopyingChainingOutput<T> extends ChainingOutput<T> {
-		
-		private final TypeSerializer<T> serializer;
-		
-		private final StreamRecord<T> copyRecord;
-
-		public CopyingChainingOutput(OneInputStreamOperator<T, ?> operator, TypeSerializer<T> serializer) {
-			super(operator);
-			this.serializer = serializer;
-			this.copyRecord = new StreamRecord<T>(null, 0L);
-		}
-
-		@Override
-		public void collect(StreamRecord<T> record) {
-			try {
-				T copy = serializer.copy(record.getValue());
-				copyRecord.replace(copy, record.getTimestamp());
-				
-				operator.setKeyContextElement(copyRecord);
-				operator.processElement(copyRecord);
-			}
-			catch (Exception e) {
-				throw new RuntimeException("Could not forward element to next operator", e);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
deleted file mode 100644
index 3d82275..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.tasks;
-
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Task for executing streaming sources.
- *
- * One important aspect of this is that the checkpointing and the emission of elements must never
- * occur at the same time. The execution must be serial. This is achieved by having the contract
- * with the StreamFunction that it must only modify its state or emit elements in
- * a synchronized block that locks on the lock Object. Also, the modification of the state
- * and the emission of elements must happen in the same block of code that is protected by the
- * synchronized block.
- *
- * @param <OUT> Type of the output elements of this source.
- */
-public class SourceStreamTask<OUT> extends StreamTask<OUT, StreamSource<OUT>> {
-
-	@Override
-	protected void init() {
-		// does not hold any resources, so no initialization needed
-	}
-
-	@Override
-	protected void cleanup() {
-		// does not hold any resources, so no cleanup needed
-	}
-	
-
-	@Override
-	protected void run() throws Exception {
-		final Object checkpointLock = getCheckpointLock();
-		final SourceOutput<StreamRecord<OUT>> output = new SourceOutput<>(getHeadOutput(), checkpointLock);
-		headOperator.run(checkpointLock, output);
-	}
-	
-	@Override
-	protected void cancelTask() throws Exception {
-		headOperator.cancel();
-	}
-
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Special output for sources that ensures that sources synchronize on  the lock object before
-	 * emitting elements.
-	 *
-	 * <p>
-	 * This is required to ensure that no concurrent method calls on operators later in the chain
-	 * can occur. When operators register a timer the timer callback is synchronized
-	 * on the same lock object.
-	 *
-	 * @param <T> The type of elements emitted by the source.
-	 */
-	private class SourceOutput<T> implements Output<T> {
-		
-		private final Output<T> output;
-		private final Object lockObject;
-
-		public SourceOutput(Output<T> output, Object lockObject) {
-			this.output = output;
-			this.lockObject = lockObject;
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			synchronized (lockObject) {
-				output.emitWatermark(mark);
-			}
-		}
-
-		@Override
-		public void collect(T record) {
-			synchronized (lockObject) {
-				checkTimerException();
-				output.collect(record);
-			}
-		}
-
-		@Override
-		public void close() {
-			output.close();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
deleted file mode 100644
index 2125df1..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.tasks;
-
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
-import org.apache.flink.streaming.runtime.io.BlockingQueueBroker;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StreamIterationHead<OUT> extends OneInputStreamTask<OUT, OUT> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamIterationHead.class);
-
-	private volatile boolean running = true;
-
-	// ------------------------------------------------------------------------
-	
-	@Override
-	protected void run() throws Exception {
-		
-		final String iterationId = getConfiguration().getIterationId();
-		if (iterationId == null || iterationId.length() == 0) {
-			throw new Exception("Missing iteration ID in the task configuration");
-		}
-		
-		final String brokerID = createBrokerIdString(getEnvironment().getJobID(), iterationId ,
-				getEnvironment().getIndexInSubtaskGroup());
-		
-		final long iterationWaitTime = getConfiguration().getIterationWaitTime();
-		final boolean shouldWait = iterationWaitTime > 0;
-
-		final BlockingQueue<StreamRecord<OUT>> dataChannel = new ArrayBlockingQueue<StreamRecord<OUT>>(1);
-
-		// offer the queue for the tail
-		BlockingQueueBroker.INSTANCE.handIn(brokerID, dataChannel);
-		LOG.info("Iteration head {} added feedback queue under {}", getName(), brokerID);
-
-		// do the work 
-		try {
-			@SuppressWarnings("unchecked")
-			RecordWriterOutput<OUT>[] outputs = (RecordWriterOutput<OUT>[]) getStreamOutputs();
-
-			// If timestamps are enabled we make sure to remove cyclic watermark dependencies
-			if (getExecutionConfig().areTimestampsEnabled()) {
-				for (RecordWriterOutput<OUT> output : outputs) {
-					output.emitWatermark(new Watermark(Long.MAX_VALUE));
-				}
-			}
-
-			while (running) {
-				StreamRecord<OUT> nextRecord = shouldWait ?
-					dataChannel.poll(iterationWaitTime, TimeUnit.MILLISECONDS) :
-					dataChannel.take();
-
-				if (nextRecord != null) {
-					for (RecordWriterOutput<OUT> output : outputs) {
-						output.collect(nextRecord);
-					}
-				}
-				else {
-					// done
-					break;
-				}
-			}
-		}
-		finally {
-			// make sure that we remove the queue from the broker, to prevent a resource leak
-			BlockingQueueBroker.INSTANCE.remove(brokerID);
-			LOG.info("Iteration head {} removed feedback queue under {}", getName(), brokerID);
-		}
-	}
-
-	@Override
-	protected void cancelTask() {
-		running = false;
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void init() {
-		// does not hold any resources, no initialization necessary
-	}
-
-	@Override
-	protected void cleanup() throws Exception {
-		// does not hold any resources, no cleanup necessary
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates the identification string with which head and tail task find the shared blocking
-	 * queue for the back channel. The identification string is unique per parallel head/tail pair
-	 * per iteration per job.
-	 * 
-	 * @param jid The job ID.
-	 * @param iterationID The id of the iteration in the job.
-	 * @param subtaskIndex The parallel subtask number
-	 * @return The identification string.
-	 */
-	public static String createBrokerIdString(JobID jid, String iterationID, int subtaskIndex) {
-		return jid + "-" + iterationID + "-" + subtaskIndex;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java
deleted file mode 100644
index 9bb5311..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java
+++ /dev/null
@@ -1,92 +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.tasks;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.io.BlockingQueueBroker;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StreamIterationTail<IN> extends OneInputStreamTask<IN, IN> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(StreamIterationTail.class);
-
-	@Override
-	public void init() throws Exception {
-		super.init();
-		
-		final String iterationId = getConfiguration().getIterationId();
-		if (iterationId == null || iterationId.length() == 0) {
-			throw new Exception("Missing iteration ID in the task configuration");
-		}
-
-		final String brokerID = StreamIterationHead.createBrokerIdString(getEnvironment().getJobID(), iterationId,
-				getEnvironment().getIndexInSubtaskGroup());
-
-		final long iterationWaitTime = getConfiguration().getIterationWaitTime();
-
-		LOG.info("Iteration tail {} trying to acquire feedback queue under {}", getName(), brokerID);
-		
-		@SuppressWarnings("unchecked")
-		BlockingQueue<StreamRecord<IN>> dataChannel =
-				(BlockingQueue<StreamRecord<IN>>) BlockingQueueBroker.INSTANCE.get(brokerID);
-		
-		LOG.info("Iteration tail {} acquired feedback queue {}", getName(), brokerID);
-		
-		this.headOperator = new RecordPusher<>(dataChannel, iterationWaitTime);
-	}
-
-	private static class RecordPusher<IN> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
-		
-		private static final long serialVersionUID = 1L;
-
-		@SuppressWarnings("NonSerializableFieldInSerializableClass")
-		private final BlockingQueue<StreamRecord<IN>> dataChannel;
-		
-		private final long iterationWaitTime;
-		
-		private final boolean shouldWait;
-
-		RecordPusher(BlockingQueue<StreamRecord<IN>> dataChannel, long iterationWaitTime) {
-			this.dataChannel = dataChannel;
-			this.iterationWaitTime = iterationWaitTime;
-			this.shouldWait =  iterationWaitTime > 0;
-		}
-
-		@Override
-		public void processElement(StreamRecord<IN> record) throws Exception {
-			if (shouldWait) {
-				dataChannel.offer(record, iterationWaitTime, TimeUnit.MILLISECONDS);
-			}
-			else {
-				dataChannel.put(record);
-			}
-		}
-
-		@Override
-		public void processWatermark(Watermark mark) {
-			// ignore
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
deleted file mode 100644
index 8c58e29..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ /dev/null
@@ -1,616 +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.tasks;
-
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobgraph.tasks.StatefulTask;
-import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
-import org.apache.flink.runtime.util.event.EventListener;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.StateBackendFactory;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Base class for all streaming tasks. A task is the unit of local processing that is deployed
- * and executed by the TaskManagers. Each task runs one or more {@link StreamOperator}s which form
- * the Task's operator chain. Operators that are chained together execute synchronously in the
- * same thread and hence on the same stream partition. A common case for these chaines
- * are successive map/flatmap/filter tasks.
- * 
- * <p>The task chain contains one "head" operator and multiple chained operators. 
- * The StreamTask is specialized for the type of the head operator: one-input and two-input tasks,
- * as well as for sources, iteration heads and iteration tails.
- * 
- * <p>The Task class deals with the setup of the streams read by the head operator, and the streams 
- * produced by the operators at the ends of the operator chain. Note that the chain may fork and
- * thus have multiple ends.
- *
- * The life cycle of the task is set up as follows: 
- * <pre>
- *  -- registerInputOutput()
- *         |
- *         +----> Create basic utils (config, etc) and load the chain of operators
- *         +----> operators.setup()
- *         +----> task specific init()
- *  
- *  -- restoreState() -> restores state of all operators in the chain
- *  
- *  -- invoke()
- *        |
- *        +----> open-operators()
- *        +----> run()
- *        +----> close-operators()
- *        +----> dispose-operators()
- *        +----> common cleanup
- *        +----> task specific cleanup()
- * </pre>
- *
- * <p> The {@code StreamTask} has a lock object called {@code lock}. All calls to methods on a
- * {@code StreamOperator} must be synchronized on this lock object to ensure that no methods
- * are called concurrently.
- * 
- * @param <OUT>
- * @param <Operator>
- */
-public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
-		extends AbstractInvokable
-		implements StatefulTask<StreamTaskStateList> {
-
-	/** The thread group that holds all trigger timer threads */
-	public static final ThreadGroup TRIGGER_THREAD_GROUP = new ThreadGroup("Triggers");
-	
-	/** The logger used by the StreamTask and its subclasses */
-	protected static final Logger LOG = LoggerFactory.getLogger(StreamTask.class);
-	
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * All interaction with the {@code StreamOperator} must be synchronized on this lock object to ensure that
-	 * we don't have concurrent method calls that void consistent checkpoints.
-	 */
-	private final Object lock = new Object();
-	
-	/** the head operator that consumes the input streams of this task */
-	protected Operator headOperator;
-
-	/** The chain of operators executed by this task */
-	private OperatorChain<OUT> operatorChain;
-	
-	/** The configuration of this streaming task */
-	private StreamConfig configuration;
-
-	/** The class loader used to load dynamic classes of a job */
-	private ClassLoader userClassLoader;
-	
-	/** The state backend that stores the state and checkpoints for this task */
-	private StateBackend<?> stateBackend;
-	
-	/** The executor service that schedules and calls the triggers of this task*/
-	private ScheduledExecutorService timerService;
-	
-	/** The map of user-defined accumulators of this task */
-	private Map<String, Accumulator<?, ?>> accumulatorMap;
-	
-	/** The state to be restored once the initialization is done */
-	private StreamTaskStateList lazyRestoreState;
-
-	/** This field is used to forward an exception that is caught in the timer thread. Subclasses
-	 * must ensure that exceptions stored here get thrown on the actual execution Thread. */
-	private volatile TimerException timerException;
-	
-	/** Flag to mark the task "in operation", in which case check
-	 * needs to be initialized to true, so that early cancel() before invoke() behaves correctly */
-	private volatile boolean isRunning;
-	
-
-	// ------------------------------------------------------------------------
-	//  Life cycle methods for specific implementations
-	// ------------------------------------------------------------------------
-
-	protected abstract void init() throws Exception;
-	
-	protected abstract void run() throws Exception;
-	
-	protected abstract void cleanup() throws Exception;
-	
-	protected abstract void cancelTask() throws Exception;
-
-	// ------------------------------------------------------------------------
-	//  Core work methods of the Stream Task
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public final void registerInputOutput() throws Exception {
-		LOG.debug("registerInputOutput for {}", getName());
-
-		boolean initializationCompleted = false;
-		try {
-			AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
-
-			userClassLoader = getUserCodeClassLoader();
-			configuration = new StreamConfig(getTaskConfiguration());
-			accumulatorMap = accumulatorRegistry.getUserMap();
-
-			stateBackend = createStateBackend();
-			stateBackend.initializeForJob(getEnvironment().getJobID());
-
-			headOperator = configuration.getStreamOperator(userClassLoader);
-			operatorChain = new OperatorChain<>(this, headOperator, accumulatorRegistry.getReadWriteReporter());
-
-			if (headOperator != null) {
-				headOperator.setup(this, configuration, operatorChain.getChainEntryPoint());
-			}
-
-			timerService = Executors.newSingleThreadScheduledExecutor(
-					new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()));
-
-			// task specific initialization
-			init();
-			
-			initializationCompleted = true;
-		}
-		finally {
-			if (!initializationCompleted) {
-				if (timerService != null) {
-					timerService.shutdownNow();
-				}
-				if (operatorChain != null) {
-					operatorChain.releaseOutputs();
-				}
-			}
-		}
-	}
-	
-	@Override
-	public final void invoke() throws Exception {
-		LOG.debug("Invoking {}", getName());
-		
-		boolean disposed = false;
-		try {
-			// first order of business is to ive operators back their state
-			restoreStateLazy();
-			
-			// we need to make sure that any triggers scheduled in open() cannot be
-			// executed before all operators are opened
-			synchronized (lock) {
-				openAllOperators();
-			}
-
-			// let the task do its work
-			isRunning = true;
-			run();
-			isRunning = false;
-			
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Finished task {}", getName());
-			}
-			
-			// make sure no further checkpoint and notification actions happen.
-			// we make sure that no other thread is currently in the locked scope before
-			// we close the operators by trying to acquire the checkpoint scope lock
-			// we also need to make sure that no triggers fire concurrently with the close logic
-			synchronized (lock) {
-				// this is part of the main logic, so if this fails, the task is considered failed
-				closeAllOperators();
-			}
-			
-			// make sure all buffered data is flushed
-			operatorChain.flushOutputs();
-
-			// make an attempt to dispose the operators such that failures in the dispose call
-			// still let the computation fail
-			tryDisposeAllOperators();
-			disposed = true;
-		}
-		finally {
-			isRunning = false;
-
-			timerService.shutdownNow();
-			
-			// release the output resources. this method should never fail.
-			if (operatorChain != null) {
-				operatorChain.releaseOutputs();
-			}
-
-			// we must! perform this cleanup
-
-			try {
-				cleanup();
-			}
-			catch (Throwable t) {
-				// catch and log the exception to not replace the original exception
-				LOG.error("Error during cleanup of stream task.");
-			}
-			
-			// if the operators were not disposed before, do a hard dispose
-			if (!disposed) {
-				disposeAllOperators();
-			}
-
-			try {
-				if (stateBackend != null) {
-					stateBackend.close();
-				}
-			} catch (Throwable t) {
-				LOG.error("Error while closing the state backend", t);
-			}
-		}
-	}
-	
-	@Override
-	public final void cancel() throws Exception {
-		isRunning = false;
-		cancelTask();
-	}
-	
-	private void openAllOperators() throws Exception {
-		for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
-			if (operator != null) {
-				operator.open();
-			}
-		}
-	}
-
-	private void closeAllOperators() throws Exception {
-		// We need to close them first to last, since upstream operators in the chain might emit
-		// elements in their close methods.
-		StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
-		for (int i = allOperators.length - 1; i >= 0; i--) {
-			StreamOperator<?> operator = allOperators[i];
-			if (operator != null) {
-				operator.close();
-			}
-		}
-	}
-
-	private void tryDisposeAllOperators() throws Exception {
-		for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
-			if (operator != null) {
-				operator.dispose();
-			}
-		}
-	}
-	
-	private void disposeAllOperators() {
-		for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
-			try {
-				if (operator != null) {
-					operator.dispose();
-				}
-			}
-			catch (Throwable t) {
-				LOG.error("Error during disposal of stream operator.", t);
-			}
-		}
-	}
-
-	/**
-	 * The finalize method shuts down the timer. This is a fail-safe shutdown, in case the original
-	 * shutdown method was never called.
-	 *
-	 * <p>
-	 * This should not be relied upon! It will cause shutdown to happen much later than if manual
-	 * shutdown is attempted, and cause threads to linger for longer than needed.
-	 */
-	@Override
-	protected void finalize() throws Throwable {
-		super.finalize();
-		if (timerService != null) {
-			if (!timerService.isTerminated()) {
-				LOG.warn("Timer service was not shut down. Shutting down in finalize().");
-			}
-			timerService.shutdown();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Access to properties and utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the name of the task, in the form "taskname (2/5)".
-	 * @return The name of the task.
-	 */
-	public String getName() {
-		return getEnvironment().getTaskNameWithSubtasks();
-	}
-
-	/**
-	 * Gets the lock object on which all operations that involve data and state mutation have to lock. 
-	 
-	 * @return The checkpoint lock object.
-	 */
-	public Object getCheckpointLock() {
-		return lock;
-	}
-	
-	public StreamConfig getConfiguration() {
-		return configuration;
-	}
-
-	public Map<String, Accumulator<?, ?>> getAccumulatorMap() {
-		return accumulatorMap;
-	}
-	
-	public Output<StreamRecord<OUT>> getHeadOutput() {
-		return operatorChain.getChainEntryPoint();
-	}
-	
-	public RecordWriterOutput<?>[] getStreamOutputs() {
-		return operatorChain.getStreamOutputs();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpoint and Restore
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public void setInitialState(StreamTaskStateList initialState) {
-		lazyRestoreState = initialState;
-	}
-	
-	public void restoreStateLazy() throws Exception {
-		if (lazyRestoreState != null) {
-			LOG.info("Restoring checkpointed state to task {}", getName());
-			
-			try {
-				final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
-				final StreamTaskState[] states = lazyRestoreState.getState(userClassLoader);
-				
-				// be GC friendly
-				lazyRestoreState = null;
-				
-				for (int i = 0; i < states.length; i++) {
-					StreamTaskState state = states[i];
-					StreamOperator<?> operator = allOperators[i];
-					
-					if (state != null && operator != null) {
-						LOG.debug("Task {} in chain ({}) has checkpointed state", i, getName());
-						operator.restoreState(state);
-					}
-					else if (operator != null) {
-						LOG.debug("Task {} in chain ({}) does not have checkpointed state", i, getName());
-					}
-				}
-			}
-			catch (Exception e) {
-				throw new Exception("Could not restore checkpointed state to operators and functions", e);
-			}
-		}
-	}
-
-	@Override
-	public void triggerCheckpoint(long checkpointId, long timestamp) throws Exception {
-		LOG.debug("Starting checkpoint {} on task {}", checkpointId, getName());
-		
-		synchronized (lock) {
-			if (isRunning) {
-
-				// since both state checkpointing and downstream barrier emission occurs in this
-				// lock scope, they are an atomic operation regardless of the order in which they occur
-				// we immediately emit the checkpoint barriers, so the downstream operators can start
-				// their checkpoint work as soon as possible
-				operatorChain.broadcastCheckpointBarrier(checkpointId, timestamp);
-				
-				// now draw the state snapshot
-				try {
-					final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
-					final StreamTaskState[] states = new StreamTaskState[allOperators.length];
-					
-					for (int i = 0; i < states.length; i++) {
-						StreamOperator<?> operator = allOperators[i];
-						if (operator != null) {
-							StreamTaskState state = operator.snapshotOperatorState(checkpointId, timestamp);
-							states[i] = state.isEmpty() ? null : state;
-						}
-					}
-
-					StreamTaskStateList allStates = new StreamTaskStateList(states);
-					if (allStates.isEmpty()) {
-						getEnvironment().acknowledgeCheckpoint(checkpointId);
-					} else {
-						getEnvironment().acknowledgeCheckpoint(checkpointId, allStates);
-					}
-				}
-				catch (Exception e) {
-					if (isRunning) {
-						throw e;
-					}
-				}
-			}
-		}
-	}
-	
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		synchronized (lock) {
-			if (isRunning) {
-				LOG.debug("Notification of complete checkpoint for task {}", getName());
-				
-				for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
-					if (operator != null) {
-						operator.notifyOfCompletedCheckpoint(checkpointId);
-					}
-				}
-			}
-			else {
-				LOG.debug("Ignoring notification of complete checkpoint for not-running task {}", getName());
-			}
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  State backend
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the state backend used by this task. The state backend defines how to maintain the
-	 * key/value state and how and where to store state snapshots.
-	 * 
-	 * @return The state backend used by this task.
-	 */
-	public StateBackend<?> getStateBackend() {
-		return stateBackend;
-	}
-	
-	private StateBackend<?> createStateBackend() throws Exception {
-		StateBackend<?> configuredBackend = configuration.getStateBackend(userClassLoader);
-
-		if (configuredBackend != null) {
-			// backend has been configured on the environment
-			LOG.info("Using user-defined state backend: " + configuredBackend);
-			return configuredBackend;
-		} else {
-			// see if we have a backend specified in the configuration
-			Configuration flinkConfig = getEnvironment().getTaskManagerInfo().getConfiguration();
-			String backendName = flinkConfig.getString(ConfigConstants.STATE_BACKEND, null);
-
-			if (backendName == null) {
-				LOG.warn("No state backend has been specified, using default state backend (Memory / JobManager)");
-				backendName = "jobmanager";
-			}
-
-			backendName = backendName.toLowerCase();
-			switch (backendName) {
-				case "jobmanager":
-					LOG.info("State backend is set to heap memory (checkpoint to jobmanager)");
-					return MemoryStateBackend.defaultInstance();
-
-				case "filesystem":
-					FsStateBackend backend = new FsStateBackendFactory().createFromConfig(flinkConfig);
-					LOG.info("State backend is set to heap memory (checkpoints to filesystem \""
-						+ backend.getBasePath() + "\")");
-					return backend;
-
-				default:
-					try {
-						@SuppressWarnings("rawtypes")
-						Class<? extends StateBackendFactory> clazz =
-							Class.forName(backendName, false, userClassLoader).asSubclass(StateBackendFactory.class);
-
-						return (StateBackend<?>) clazz.newInstance();
-					} catch (ClassNotFoundException e) {
-						throw new IllegalConfigurationException("Cannot find configured state backend: " + backendName);
-					} catch (ClassCastException e) {
-						throw new IllegalConfigurationException("The class configured under '" +
-							ConfigConstants.STATE_BACKEND + "' is not a valid state backend factory (" +
-							backendName + ')');
-					} catch (Throwable t) {
-						throw new IllegalConfigurationException("Cannot create configured state backend", t);
-					}
-			}
-		}
-	}
-
-	/**
-	 * Registers a timer.
-	 */
-	public void registerTimer(final long timestamp, final Triggerable target) {
-		long delay = Math.max(timestamp - System.currentTimeMillis(), 0);
-
-		timerService.schedule(
-				new TriggerTask(this, lock, target, timestamp),
-				delay,
-				TimeUnit.MILLISECONDS);
-	}
-	
-	public void checkTimerException() throws TimerException {
-		if (timerException != null) {
-			throw timerException;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return getName();
-	}
-
-	protected final EventListener<CheckpointBarrier> getCheckpointBarrierListener() {
-		return new EventListener<CheckpointBarrier>() {
-			@Override
-			public void onEvent(CheckpointBarrier barrier) {
-				try {
-					triggerCheckpoint(barrier.getId(), barrier.getTimestamp());
-				}
-				catch (Exception e) {
-					throw new RuntimeException("Error triggering a checkpoint as the result of receiving checkpoint barrier", e);
-				}
-			}
-		};
-	}
-	
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Internal task that is invoked by the timer service and triggers the target.
-	 */
-	private static final class TriggerTask implements Runnable {
-
-		private final Object lock;
-		private final Triggerable target;
-		private final long timestamp;
-		private final StreamTask<?, ?> task;
-
-		TriggerTask(StreamTask<?, ?> task, final Object lock, Triggerable target, long timestamp) {
-			this.task = task;
-			this.lock = lock;
-			this.target = target;
-			this.timestamp = timestamp;
-		}
-
-		@Override
-		public void run() {
-			synchronized (lock) {
-				try {
-					target.trigger(timestamp);
-				} catch (Throwable t) {
-					LOG.error("Caught exception while processing timer.", t);
-					if (task.timerException == null) {
-						task.timerException = new TimerException(t);
-					}
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskException.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskException.java
deleted file mode 100644
index 5680810..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskException.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.runtime.tasks;
-
-/**
- * An exception that is thrown by the stream vertices when encountering an
- * illegal condition.
- */
-public class StreamTaskException extends RuntimeException {
-
-	/**
-	 * Serial version UID for serialization interoperability.
-	 */
-	private static final long serialVersionUID = 8392043527067472439L;
-
-	/**
-	 * Creates a compiler exception with no message and no cause.
-	 */
-	public StreamTaskException() {
-	}
-
-	/**
-	 * Creates a compiler exception with the given message and no cause.
-	 * 
-	 * @param message
-	 *            The message for the exception.
-	 */
-	public StreamTaskException(String message) {
-		super(message);
-	}
-
-	/**
-	 * Creates a compiler exception with the given cause and no message.
-	 * 
-	 * @param cause
-	 *            The <tt>Throwable</tt> that caused this exception.
-	 */
-	public StreamTaskException(Throwable cause) {
-		super(cause);
-	}
-
-	/**
-	 * Creates a compiler exception with the given message and cause.
-	 * 
-	 * @param message
-	 *            The message for the exception.
-	 * @param cause
-	 *            The <tt>Throwable</tt> that caused this exception.
-	 */
-	public StreamTaskException(String message, Throwable cause) {
-		super(message, cause);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
deleted file mode 100644
index afeabd9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.tasks;
-
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.KvStateSnapshot;
-
-import java.io.Serializable;
-import java.util.ConcurrentModificationException;
-import java.util.HashMap;
-import java.util.Iterator;
-
-/**
- * The state checkpointed by a {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator}.
- * This state consists of any combination of those three:
- * <ul>
- *     <li>The state of the stream operator, if it implements the Checkpointed interface.</li>
- *     <li>The state of the user function, if it implements the Checkpointed interface.</li>
- *     <li>The key/value state of the operator, if it executes on a KeyedDataStream.</li>
- * </ul>
- */
-public class StreamTaskState implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-	
-	private StateHandle<?> operatorState;
-
-	private StateHandle<Serializable> functionState;
-
-	private HashMap<String, KvStateSnapshot<?, ?, ?>> kvStates;
-
-	// ------------------------------------------------------------------------
-
-	public StateHandle<?> getOperatorState() {
-		return operatorState;
-	}
-
-	public void setOperatorState(StateHandle<?> operatorState) {
-		this.operatorState = operatorState;
-	}
-
-	public StateHandle<Serializable> getFunctionState() {
-		return functionState;
-	}
-
-	public void setFunctionState(StateHandle<Serializable> functionState) {
-		this.functionState = functionState;
-	}
-
-	public HashMap<String, KvStateSnapshot<?, ?, ?>> getKvStates() {
-		return kvStates;
-	}
-
-	public void setKvStates(HashMap<String, KvStateSnapshot<?, ?, ?>> kvStates) {
-		this.kvStates = kvStates;
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Checks if this state object actually contains any state, or if all of the state
-	 * fields are null.
-	 * 
-	 * @return True, if all state is null, false if at least one state is not null.
-	 */
-	public boolean isEmpty() {
-		return operatorState == null & functionState == null & kvStates == null;
-	}
-
-	/**
-	 * Discards all the contained states and sets them to null.
-	 * 
-	 * @throws Exception Forwards exceptions that occur when releasing the
-	 *                   state handles and snapshots.
-	 */
-	public void discardState() throws Exception {
-		StateHandle<?> operatorState = this.operatorState;
-		StateHandle<?> functionState = this.functionState;
-		HashMap<String, KvStateSnapshot<?, ?, ?>> kvStates = this.kvStates;
-		
-		if (operatorState != null) {
-			operatorState.discardState();
-		}
-		if (functionState != null) {
-			functionState.discardState();
-		}
-		if (kvStates != null) {
-			while (kvStates.size() > 0) {
-				try {
-					Iterator<KvStateSnapshot<?, ?, ?>> values = kvStates.values().iterator();
-					while (values.hasNext()) {
-						KvStateSnapshot<?, ?, ?> s = values.next();
-						s.discardState();
-						values.remove();
-					}
-				}
-				catch (ConcurrentModificationException e) {
-					// fall through the loop
-				}
-			}
-		}
-
-		this.operatorState = null;
-		this.functionState = null;
-		this.kvStates = null;
-	}
-}
- 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java
deleted file mode 100644
index 7b8dbd5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.tasks;
-
-import org.apache.flink.runtime.state.StateHandle;
-
-/**
- * List of task states for a chain of streaming tasks.
- */
-public class StreamTaskStateList implements StateHandle<StreamTaskState[]> {
-
-	private static final long serialVersionUID = 1L;
-
-	/** The states for all operator */
-	private final StreamTaskState[] states;
-
-	
-	public StreamTaskStateList(StreamTaskState[] states) {
-		this.states = states;
-	}
-
-	public boolean isEmpty() {
-		for (StreamTaskState state : states) {
-			if (state != null) {
-				return false;
-			}
-		}
-		return true;
-	}
-	
-	@Override
-	public StreamTaskState[] getState(ClassLoader userCodeClassLoader) {
-		return states;
-	}
-
-	@Override
-	public void discardState() throws Exception {
-		for (StreamTaskState state : states) {
-			if (state != null) {
-				state.discardState();
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerException.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerException.java
deleted file mode 100644
index 3e1c1e5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerException.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.runtime.tasks;
-
-/**
- * {@code RuntimeException} for wrapping exceptions that are thrown in the timer callback of
- * the timer service in {@link StreamTask}.
- */
-public class TimerException extends RuntimeException {
-	private static final long serialVersionUID = 1L;
-
-	public TimerException(Throwable cause) {
-		super(cause);
-	}
-
-	@Override
-	public String toString() {
-		return "TimerException{" + getCause() + "}";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
deleted file mode 100644
index d2d8a2e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.tasks;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor;
-
-public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputStreamOperator<IN1, IN2, OUT>> {
-
-	private StreamTwoInputProcessor<IN1, IN2> inputProcessor;
-	
-	private volatile boolean running = true;
-
-	@Override
-	public void init() throws Exception {
-		StreamConfig configuration = getConfiguration();
-		ClassLoader userClassLoader = getUserCodeClassLoader();
-		
-		TypeSerializer<IN1> inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader);
-		TypeSerializer<IN2> inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader);
-	
-		int numberOfInputs = configuration.getNumberOfInputs();
-	
-		ArrayList<InputGate> inputList1 = new ArrayList<InputGate>();
-		ArrayList<InputGate> inputList2 = new ArrayList<InputGate>();
-	
-		List<StreamEdge> inEdges = configuration.getInPhysicalEdges(userClassLoader);
-	
-		for (int i = 0; i < numberOfInputs; i++) {
-			int inputType = inEdges.get(i).getTypeNumber();
-			InputGate reader = getEnvironment().getInputGate(i);
-			switch (inputType) {
-				case 1:
-					inputList1.add(reader);
-					break;
-				case 2:
-					inputList2.add(reader);
-					break;
-				default:
-					throw new RuntimeException("Invalid input type number: " + inputType);
-			}
-		}
-	
-		this.inputProcessor = new StreamTwoInputProcessor<IN1, IN2>(inputList1, inputList2,
-				inputDeserializer1, inputDeserializer2,
-				getCheckpointBarrierListener(),
-				configuration.getCheckpointMode(),
-				getEnvironment().getIOManager(),
-				getExecutionConfig().areTimestampsEnabled());
-
-		// make sure that stream tasks report their I/O statistics
-		AccumulatorRegistry registry = getEnvironment().getAccumulatorRegistry();
-		AccumulatorRegistry.Reporter reporter = registry.getReadWriteReporter();
-		this.inputProcessor.setReporter(reporter);
-	}
-
-	@Override
-	protected void run() throws Exception {
-		// cache some references on the stack, to make the code more JIT friendly
-		final TwoInputStreamOperator<IN1, IN2, OUT> operator = this.headOperator;
-		final StreamTwoInputProcessor<IN1, IN2> inputProcessor = this.inputProcessor;
-		final Object lock = getCheckpointLock();
-		
-		while (running && inputProcessor.processInput(operator, lock)) {
-			checkTimerException();
-		}
-	}
-
-	@Override
-	protected void cleanup() throws Exception {
-		inputProcessor.cleanup();
-	}
-
-	@Override
-	protected void cancelTask() {
-		running = false;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/package-info.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/package-info.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/package-info.java
deleted file mode 100644
index a40ae3a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/package-info.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * This package contains classes that realize streaming tasks. These tasks are
- * executable stream consumers and producers that are scheduled by the distributed
- * dataflow runtime. Each task occupies one execution slot and is run with by an
- * executing thread.
- * <p>
- * The tasks merely set up the distributed stream coordination and the checkpointing.
- * Internally, the tasks create one or more operators, perform the stream transformations.
- */
-package org.apache.flink.streaming.runtime.tasks;
\ No newline at end of file


[15/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
deleted file mode 100644
index 6855e00..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ /dev/null
@@ -1,767 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.functions.{FilterFunction, FlatMapFunction, MapFunction, Partitioner, RichFilterFunction, RichFlatMapFunction, RichMapFunction}
-import org.apache.flink.api.common.io.OutputFormat
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.api.java.tuple.{Tuple => JavaTuple}
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.api.scala.operators.ScalaCsvOutputFormat
-import org.apache.flink.core.fs.{FileSystem, Path}
-import org.apache.flink.streaming.api.collector.selector.OutputSelector
-import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWindowedStream, DataStream => JavaStream, KeyedStream => JavaKeyedStream, _}
-import org.apache.flink.streaming.api.functions.sink.SinkFunction
-import org.apache.flink.streaming.api.functions.{AscendingTimestampExtractor, TimestampExtractor}
-import org.apache.flink.streaming.api.scala.function.StatefulFunction
-import org.apache.flink.streaming.api.windowing.assigners._
-import org.apache.flink.streaming.api.windowing.time.AbstractTime
-import org.apache.flink.streaming.api.windowing.windows.{GlobalWindow, TimeWindow, Window}
-import org.apache.flink.streaming.util.serialization.SerializationSchema
-import org.apache.flink.util.Collector
-
-import scala.collection.JavaConverters._
-import scala.reflect.ClassTag
-
-class DataStream[T](javaStream: JavaStream[T]) {
-
-  /**
-   * Gets the underlying java DataStream object.
-   */
-  def getJavaStream: JavaStream[T] = javaStream
-
-  /**
-   * Returns the ID of the DataStream.
-   *
-   * @return ID of the DataStream
-   */
-  def getId = javaStream.getId
-
-  /**
-   * Returns the TypeInformation for the elements of this DataStream.
-   */
-  def getType(): TypeInformation[T] = javaStream.getType()
-
-  /**
-   * Sets the parallelism of this operation. This must be at least 1.
-   */
-  def setParallelism(parallelism: Int): DataStream[T] = {
-    javaStream match {
-      case ds: SingleOutputStreamOperator[_, _] => ds.setParallelism(parallelism)
-      case _ =>
-        throw new UnsupportedOperationException("Operator " + javaStream.toString +  " cannot " +
-          "have " +
-          "parallelism.")
-    }
-    this
-  }
-
-  /**
-   * Returns the parallelism of this operation.
-   */
-  def getParallelism = javaStream.getParallelism
-  
-  /**
-   * Returns the execution config.
-   */
-  def getExecutionConfig = javaStream.getExecutionConfig
-
-  /**
-   * Gets the name of the current data stream. This name is
-   * used by the visualization and logging during runtime.
-   *
-   * @return Name of the stream.
-   */
-  def getName : String = javaStream match {
-    case stream : SingleOutputStreamOperator[T,_] => stream.getName
-    case _ => throw new
-        UnsupportedOperationException("Only supported for operators.")
-  }
-
-  /**
-   * Sets the name of the current data stream. This name is
-   * used by the visualization and logging during runtime.
-   *
-   * @return The named operator
-   */
-  def name(name: String) : DataStream[T] = javaStream match {
-    case stream : SingleOutputStreamOperator[T,_] => stream.name(name)
-    case _ => throw new UnsupportedOperationException("Only supported for operators.")
-    this
-  }
-  
-  /**
-   * Turns off chaining for this operator so thread co-location will not be
-   * used as an optimization. </p> Chaining can be turned off for the whole
-   * job by [[StreamExecutionEnvironment.disableOperatorChaining()]]
-   * however it is not advised for performance considerations.
-   * 
-   */
-  def disableChaining(): DataStream[T] = {
-    javaStream match {
-      case ds: SingleOutputStreamOperator[_, _] => ds.disableChaining();
-      case _ =>
-        throw new UnsupportedOperationException("Only supported for operators.")
-    }
-    this
-  }
-  
-  /**
-   * Starts a new task chain beginning at this operator. This operator will
-   * not be chained (thread co-located for increased performance) to any
-   * previous tasks even if possible.
-   * 
-   */
-  def startNewChain(): DataStream[T] = {
-    javaStream match {
-      case ds: SingleOutputStreamOperator[_, _] => ds.startNewChain();
-      case _ =>
-        throw new UnsupportedOperationException("Only supported for operators.")
-    }
-    this
-  }
-  
-  /**
-   * Isolates the operator in its own resource group. This will cause the
-   * operator to grab as many task slots as its degree of parallelism. If
-   * there are no free resources available, the job will fail to start.
-   * All subsequent operators are assigned to the default resource group.
-   * 
-   */
-  def isolateResources(): DataStream[T] = {
-    javaStream match {
-      case ds: SingleOutputStreamOperator[_, _] => ds.isolateResources();
-      case _ =>
-        throw new UnsupportedOperationException("Only supported for operators.")
-    }
-    this
-  }
-  
-  /**
-   * By default all operators in a streaming job share the same resource
-   * group. Each resource group takes as many task manager slots as the
-   * maximum parallelism operator in that group. By calling this method, this
-   * operators starts a new resource group and all subsequent operators will
-   * be added to this group unless specified otherwise. Please note that
-   * local executions have by default as many available task slots as the
-   * environment parallelism, so in order to start a new resource group the
-   * degree of parallelism for the operators must be decreased from the
-   * default.
-   */
-  def startNewResourceGroup(): DataStream[T] = {
-    javaStream match {
-      case ds: SingleOutputStreamOperator[_, _] => ds.startNewResourceGroup();
-      case _ =>
-        throw new UnsupportedOperationException("Only supported for operators.")
-    }
-    this
-  }
-
-  /**
-   * Sets the maximum time frequency (ms) for the flushing of the output
-   * buffer. By default the output buffers flush only when they are full.
-   *
-   * @param timeoutMillis
-   * The maximum time between two output flushes.
-   * @return The operator with buffer timeout set.
-   */
-  def setBufferTimeout(timeoutMillis: Long): DataStream[T] = {
-    javaStream match {
-      case ds: SingleOutputStreamOperator[_, _] => ds.setBufferTimeout(timeoutMillis);
-      case _ =>
-        throw new UnsupportedOperationException("Only supported for operators.")
-    }
-    this
-  }
-
-  /**
-   * Creates a new DataStream by merging DataStream outputs of
-   * the same type with each other. The DataStreams merged using this operator
-   * will be transformed simultaneously.
-   *
-   */
-  def union(dataStreams: DataStream[T]*): DataStream[T] =
-    javaStream.union(dataStreams.map(_.getJavaStream): _*)
-
-  /**
-   * Creates a new ConnectedStreams by connecting
-   * DataStream outputs of different type with each other. The
-   * DataStreams connected using this operators can be used with CoFunctions.
-   */
-  def connect[T2](dataStream: DataStream[T2]): ConnectedStreams[T, T2] =
-    javaStream.connect(dataStream.getJavaStream)
-  
-  /**
-   * Groups the elements of a DataStream by the given key positions (for tuple/array types) to
-   * be used with grouped operators like grouped reduce or grouped aggregations.
-   */
-  def keyBy(fields: Int*): KeyedStream[T, JavaTuple] = javaStream.keyBy(fields: _*)
-
-  /**
-   * Groups the elements of a DataStream by the given field expressions to
-   * be used with grouped operators like grouped reduce or grouped aggregations.
-   */
-  def keyBy(firstField: String, otherFields: String*): KeyedStream[T, JavaTuple] =
-   javaStream.keyBy(firstField +: otherFields.toArray: _*)   
-  
-  /**
-   * Groups the elements of a DataStream by the given K key to
-   * be used with grouped operators like grouped reduce or grouped aggregations.
-   */
-  def keyBy[K: TypeInformation](fun: T => K): KeyedStream[T, K] = {
-
-    val cleanFun = clean(fun)
-    val keyType: TypeInformation[K] = implicitly[TypeInformation[K]]
-    
-    val keyExtractor = new KeySelector[T, K] with ResultTypeQueryable[K] {
-      def getKey(in: T) = cleanFun(in)
-      override def getProducedType: TypeInformation[K] = keyType
-    }
-    new JavaKeyedStream(javaStream, keyExtractor, keyType)
-  }
-
-  /**
-   * Partitions the elements of a DataStream by the given key positions (for tuple/array types) to
-   * be used with grouped operators like grouped reduce or grouped aggregations.
-   */
-  def partitionByHash(fields: Int*): DataStream[T] = javaStream.partitionByHash(fields: _*)
-
-  /**
-   * Groups the elements of a DataStream by the given field expressions to
-   * be used with grouped operators like grouped reduce or grouped aggregations.
-   */
-  def partitionByHash(firstField: String, otherFields: String*): DataStream[T] =
-    javaStream.partitionByHash(firstField +: otherFields.toArray: _*)
-
-  /**
-   * Groups the elements of a DataStream by the given K key to
-   * be used with grouped operators like grouped reduce or grouped aggregations.
-   */
-  def partitionByHash[K: TypeInformation](fun: T => K): DataStream[T] = {
-
-    val cleanFun = clean(fun)
-    val keyExtractor = new KeySelector[T, K] with ResultTypeQueryable[K] {
-      def getKey(in: T) = cleanFun(in)
-      override def getProducedType: TypeInformation[K] = implicitly[TypeInformation[K]]
-    }
-    javaStream.partitionByHash(keyExtractor)
-  }
-
-  /**
-   * Partitions a tuple DataStream on the specified key fields using a custom partitioner.
-   * This method takes the key position to partition on, and a partitioner that accepts the key
-   * type.
-   * <p>
-   * Note: This method works only on single field keys.
-   */
-  def partitionCustom[K: TypeInformation](partitioner: Partitioner[K], field: Int) : DataStream[T] =
-    javaStream.partitionCustom(partitioner, field)
-
-  /**
-   * Partitions a POJO DataStream on the specified key fields using a custom partitioner.
-   * This method takes the key expression to partition on, and a partitioner that accepts the key
-   * type.
-   * <p>
-   * Note: This method works only on single field keys.
-   */
-  def partitionCustom[K: TypeInformation](partitioner: Partitioner[K], field: String)
-  : DataStream[T] = javaStream.partitionCustom(partitioner, field)
-
-  /**
-   * Partitions a DataStream on the key returned by the selector, using a custom partitioner.
-   * This method takes the key selector to get the key to partition on, and a partitioner that
-   * accepts the key type.
-   * <p>
-   * Note: This method works only on single field keys, i.e. the selector cannot return tuples
-   * of fields.
-   */
-  def partitionCustom[K: TypeInformation](partitioner: Partitioner[K], fun: T => K)
-  : DataStream[T] = {
-    val cleanFun = clean(fun)
-    val keyExtractor = new KeySelector[T, K] with ResultTypeQueryable[K] {
-      def getKey(in: T) = cleanFun(in)
-      override def getProducedType: TypeInformation[K] = implicitly[TypeInformation[K]]
-    }
-    javaStream.partitionCustom(partitioner, keyExtractor)
-  }
-
-  /**
-   * Sets the partitioning of the DataStream so that the output tuples
-   * are broad casted to every parallel instance of the next component. This
-   * setting only effects the how the outputs will be distributed between the
-   * parallel instances of the next processing operator.
-   *
-   */
-  def broadcast: DataStream[T] = javaStream.broadcast()
-
-  /**
-   * Sets the partitioning of the DataStream so that the output values all go to 
-   * the first instance of the next processing operator. Use this setting with care
-   * since it might cause a serious performance bottleneck in the application.
-   */
-  def global: DataStream[T] = javaStream.global()
-
-  /**
-   * Sets the partitioning of the DataStream so that the output tuples
-   * are shuffled to the next component. This setting only effects the how the
-   * outputs will be distributed between the parallel instances of the next
-   * processing operator.
-   *
-   */
-  def shuffle: DataStream[T] = javaStream.shuffle()
-
-  /**
-   * Sets the partitioning of the DataStream so that the output tuples
-   * are forwarded to the local subtask of the next component (whenever
-   * possible). This is the default partitioner setting. This setting only
-   * effects the how the outputs will be distributed between the parallel
-   * instances of the next processing operator.
-   *
-   */
-  def forward: DataStream[T] = javaStream.forward()
-
-  /**
-   * Sets the partitioning of the DataStream so that the output tuples
-   * are distributed evenly to the next component.This setting only effects
-   * the how the outputs will be distributed between the parallel instances of
-   * the next processing operator.
-   *
-   */
-  def rebalance: DataStream[T] = javaStream.rebalance()
-
-  /**
-   * Initiates an iterative part of the program that creates a loop by feeding
-   * back data streams. To create a streaming iteration the user needs to define
-   * a transformation that creates two DataStreams. The first one is the output
-   * that will be fed back to the start of the iteration and the second is the output
-   * stream of the iterative part.
-   * <p>
-   * stepfunction: initialStream => (feedback, output)
-   * <p>
-   * A common pattern is to use output splitting to create feedback and output DataStream.
-   * Please refer to the .split(...) method of the DataStream
-   * <p>
-   * By default a DataStream with iteration will never terminate, but the user
-   * can use the maxWaitTime parameter to set a max waiting time for the iteration head.
-   * If no data received in the set time the stream terminates.
-   * <p>
-   * By default the feedback partitioning is set to match the input, to override this set 
-   * the keepPartitioning flag to true
-   *
-   */
-  def iterate[R](stepFunction: DataStream[T] => (DataStream[T], DataStream[R]),
-                    maxWaitTimeMillis:Long = 0,
-                    keepPartitioning: Boolean = false) : DataStream[R] = {
-    val iterativeStream = javaStream.iterate(maxWaitTimeMillis)
-
-    val (feedback, output) = stepFunction(new DataStream[T](iterativeStream))
-    iterativeStream.closeWith(feedback.getJavaStream)
-    output
-  }
-  
-  /**
-   * Initiates an iterative part of the program that creates a loop by feeding
-   * back data streams. To create a streaming iteration the user needs to define
-   * a transformation that creates two DataStreams. The first one is the output
-   * that will be fed back to the start of the iteration and the second is the output
-   * stream of the iterative part.
-   * 
-   * The input stream of the iterate operator and the feedback stream will be treated
-   * as a ConnectedStreams where the the input is connected with the feedback stream.
-   * 
-   * This allows the user to distinguish standard input from feedback inputs.
-   * 
-   * <p>
-   * stepfunction: initialStream => (feedback, output)
-   * <p>
-   * The user must set the max waiting time for the iteration head.
-   * If no data received in the set time the stream terminates. If this parameter is set
-   * to 0 then the iteration sources will indefinitely, so the job must be killed to stop.
-   *
-   */
-  def iterate[R, F: TypeInformation: ClassTag](stepFunction: ConnectedStreams[T, F] =>
-    (DataStream[F], DataStream[R]), maxWaitTimeMillis:Long): DataStream[R] = {
-    val feedbackType: TypeInformation[F] = implicitly[TypeInformation[F]]
-    val connectedIterativeStream = javaStream.iterate(maxWaitTimeMillis).
-                                   withFeedbackType(feedbackType)
-
-    val (feedback, output) = stepFunction(connectedIterativeStream)
-    connectedIterativeStream.closeWith(feedback.getJavaStream)
-    output
-  }  
-
-  /**
-   * Creates a new DataStream by applying the given function to every element of this DataStream.
-   */
-  def map[R: TypeInformation: ClassTag](fun: T => R): DataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("Map function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val mapper = new MapFunction[T, R] {
-      def map(in: T): R = cleanFun(in)
-    }
-    
-    map(mapper)
-  }
-
-  /**
-   * Creates a new DataStream by applying the given function to every element of this DataStream.
-   */
-  def map[R: TypeInformation: ClassTag](mapper: MapFunction[T, R]): DataStream[R] = {
-    if (mapper == null) {
-      throw new NullPointerException("Map function must not be null.")
-    }
-
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]
-    javaStream.map(mapper).returns(outType).asInstanceOf[JavaStream[R]]
-  }
-  
-  /**
-   * Creates a new DataStream by applying the given function to every element and flattening
-   * the results.
-   */
-  def flatMap[R: TypeInformation: ClassTag](flatMapper: FlatMapFunction[T, R]): DataStream[R] = {
-    if (flatMapper == null) {
-      throw new NullPointerException("FlatMap function must not be null.")
-    }
-    
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]
-    javaStream.flatMap(flatMapper).returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Creates a new DataStream by applying the given function to every element and flattening
-   * the results.
-   */
-  def flatMap[R: TypeInformation: ClassTag](fun: (T, Collector[R]) => Unit): DataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("FlatMap function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val flatMapper = new FlatMapFunction[T, R] {
-      def flatMap(in: T, out: Collector[R]) { cleanFun(in, out) }
-    }
-    flatMap(flatMapper)
-  }
-
-  /**
-   * Creates a new DataStream by applying the given function to every element and flattening
-   * the results.
-   */
-  def flatMap[R: TypeInformation: ClassTag](fun: T => TraversableOnce[R]): DataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("FlatMap function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val flatMapper = new FlatMapFunction[T, R] {
-      def flatMap(in: T, out: Collector[R]) { cleanFun(in) foreach out.collect }
-    }
-    flatMap(flatMapper)
-  }
-
-  /**
-   * Creates a new DataStream that contains only the elements satisfying the given filter predicate.
-   */
-  def filter(filter: FilterFunction[T]): DataStream[T] = {
-    if (filter == null) {
-      throw new NullPointerException("Filter function must not be null.")
-    }
-    javaStream.filter(filter)
-  }
-
-  /**
-   * Creates a new DataStream that contains only the elements satisfying the given filter predicate.
-   */
-  def filter(fun: T => Boolean): DataStream[T] = {
-    if (fun == null) {
-      throw new NullPointerException("Filter function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val filter = new FilterFunction[T] {
-      def filter(in: T) = cleanFun(in)
-    }
-    this.filter(filter)
-  }
-
-  /**
-   * Windows this DataStream into tumbling time windows.
-   *
-   * This is a shortcut for either `.window(TumblingTimeWindows.of(size))` or
-   * `.window(TumblingProcessingTimeWindows.of(size))` depending on the time characteristic
-   * set using
-   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic]].
-   *
-   * 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 size The size of the window.
-   */
-  def timeWindowAll(size: AbstractTime): AllWindowedStream[T, TimeWindow] = {
-    val assigner = TumblingTimeWindows.of(size).asInstanceOf[WindowAssigner[T, TimeWindow]]
-    windowAll(assigner)
-  }
-
-  /**
-   * Windows this DataStream into sliding time windows.
-   *
-   * This is a shortcut for either `.window(SlidingTimeWindows.of(size, slide))` or
-   * `.window(SlidingProcessingTimeWindows.of(size, slide))` depending on the time characteristic
-   * set using
-   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic]].
-   *
-   * 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 size The size of the window.
-   */
-  def timeWindowAll(size: AbstractTime, slide: AbstractTime): AllWindowedStream[T, TimeWindow] = {
-    val assigner = SlidingTimeWindows.of(size, slide).asInstanceOf[WindowAssigner[T, TimeWindow]]
-    windowAll(assigner)
-  }
-
-  /**
-   * Windows this [[DataStream]] into sliding count windows.
-   *
-   * 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 size The size of the windows in number of elements.
-   * @param slide The slide interval in number of elements.
-   */
-  def countWindowAll(size: Long, slide: Long): AllWindowedStream[T, GlobalWindow] = {
-    new AllWindowedStream(javaStream.countWindowAll(size, slide))
-  }
-
-  /**
-   * Windows this [[DataStream]] into tumbling count windows.
-   *
-   * 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 size The size of the windows in number of elements.
-   */
-  def countWindowAll(size: Long): AllWindowedStream[T, GlobalWindow] = {
-    new AllWindowedStream(javaStream.countWindowAll(size))
-  }
-
-  /**
-   * Windows this data stream to a [[AllWindowedStream]], which evaluates windows
-   * over a key grouped stream. Elements are put into windows by a [[WindowAssigner]]. The grouping
-   * of elements is done both by key and by window.
-   *
-   * A [[org.apache.flink.streaming.api.windowing.triggers.Trigger]] can be defined to specify
-   * when windows are evaluated. However, `WindowAssigner` have a default `Trigger`
-   * that is used if a `Trigger` is not specified.
-   *
-   * 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 `WindowAssigner` that assigns elements to windows.
-   * @return The trigger windows data stream.
-   */
-  def windowAll[W <: Window](assigner: WindowAssigner[_ >: T, W]): AllWindowedStream[T, W] = {
-    new AllWindowedStream[T, W](new JavaAllWindowedStream[T, W](javaStream, assigner))
-  }
-  /**
-   * Extracts a timestamp from an element and assigns it as the internal timestamp of that element.
-   * The internal timestamps are, for example, used to to event-time window operations.
-   *
-   * If you know that the timestamps are strictly increasing you can use an
-   * [[org.apache.flink.streaming.api.functions.AscendingTimestampExtractor]]. Otherwise,
-   * you should provide a [[TimestampExtractor]] that also implements
-   * [[TimestampExtractor#getCurrentWatermark]] to keep track of watermarks.
-   *
-   * @see org.apache.flink.streaming.api.watermark.Watermark
-   */
-  def assignTimestamps(extractor: TimestampExtractor[T]): DataStream[T] = {
-    javaStream.assignTimestamps(clean(extractor))
-  }
-
-  /**
-   * Extracts a timestamp from an element and assigns it as the internal timestamp of that element.
-   * The internal timestamps are, for example, used to to event-time window operations.
-   *
-   * If you know that the timestamps are strictly increasing you can use an
-   * [[org.apache.flink.streaming.api.functions.AscendingTimestampExtractor]]. Otherwise,
-   * you should provide a [[TimestampExtractor]] that also implements
-   * [[TimestampExtractor#getCurrentWatermark]] to keep track of watermarks.
-   *
-   * @see org.apache.flink.streaming.api.watermark.Watermark
-   */
-  def assignAscendingTimestamps(extractor: T => Long): DataStream[T] = {
-    val cleanExtractor = clean(extractor)
-    val extractorFunction = new AscendingTimestampExtractor[T] {
-      def extractAscendingTimestamp(element: T, currentTimestamp: Long): Long = {
-        cleanExtractor(element)
-      }
-    }
-    javaStream.assignTimestamps(extractorFunction)
-  }
-
-  /**
-   *
-   * Operator used for directing tuples to specific named outputs using an
-   * OutputSelector. Calling this method on an operator creates a new
-   * [[SplitStream]].
-   */
-  def split(selector: OutputSelector[T]): SplitStream[T] = javaStream.split(selector)
-
-  /**
-   * Creates a new [[SplitStream]] that contains only the elements satisfying the
-   *  given output selector predicate.
-   */
-  def split(fun: T => TraversableOnce[String]): SplitStream[T] = {
-    if (fun == null) {
-      throw new NullPointerException("OutputSelector must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val selector = new OutputSelector[T] {
-      def select(in: T): java.lang.Iterable[String] = {
-        cleanFun(in).toIterable.asJava
-      }
-    }
-    split(selector)
-  }
-
-  /**
-   * Creates a co-group operation. See [[CoGroupedStreams]] for an example of how the keys
-   * and window can be specified.
-   */
-  def coGroup[T2](otherStream: DataStream[T2]): CoGroupedStreams.Unspecified[T, T2] = {
-    CoGroupedStreams.createCoGroup(this, otherStream)
-  }
-
-  /**
-   * Creates a join operation. See [[JoinedStreams]] for an example of how the keys
-   * and window can be specified.
-   */
-  def join[T2](otherStream: DataStream[T2]): JoinedStreams.Unspecified[T, T2] = {
-    JoinedStreams.createJoin(this, otherStream)
-  }
-
-  /**
-   * Writes a DataStream to the standard output stream (stdout). For each
-   * element of the DataStream the result of .toString is
-   * written.
-   *
-   */
-  def print(): DataStreamSink[T] = javaStream.print()
-
-  /**
-   * Writes a DataStream to the standard output stream (stderr).
-   * 
-   * For each element of the DataStream the result of
-   * [[AnyRef.toString()]] is written.
-   *
-   * @return The closed DataStream.
-   */
-  def printToErr() = javaStream.printToErr()
-
-  /**
-   * Writes a DataStream to the file specified by path in text format. The
-   * writing is performed periodically, in every millis milliseconds. For
-   * every element of the DataStream the result of .toString
-   * is written.
-   *
-   */
-  def writeAsText(path: String, millis: Long = 0): DataStreamSink[T] =
-    javaStream.writeAsText(path, millis)
-
-  /**
-   * Writes a DataStream to the file specified by path in text format. The
-   * writing is performed periodically, in every millis milliseconds. For
-   * every element of the DataStream the result of .toString
-   * is written.
-   *
-   */
-  def writeAsCsv(
-      path: String,
-      millis: Long = 0,
-      rowDelimiter: String = ScalaCsvOutputFormat.DEFAULT_LINE_DELIMITER,
-      fieldDelimiter: String = ScalaCsvOutputFormat.DEFAULT_FIELD_DELIMITER,
-      writeMode: FileSystem.WriteMode = null): DataStreamSink[T] = {
-    require(javaStream.getType.isTupleType, "CSV output can only be used with Tuple DataSets.")
-    val of = new ScalaCsvOutputFormat[Product](new Path(path), rowDelimiter, fieldDelimiter)
-    if (writeMode != null) {
-      of.setWriteMode(writeMode)
-    }
-    javaStream.write(of.asInstanceOf[OutputFormat[T]], millis)
-  }
-
-  /**
-   * Writes a DataStream using the given [[OutputFormat]]. The
-   * writing is performed periodically, in every millis milliseconds.
-   */
-  def write(format: OutputFormat[T], millis: Long): DataStreamSink[T] = {
-    javaStream.write(format, millis)
-  }
-
-  /**
-   * Writes the DataStream to a socket as a byte array. The format of the output is
-   * specified by a [[SerializationSchema]].
-   */
-  def writeToSocket(
-      hostname: String,
-      port: Integer,
-      schema: SerializationSchema[T, Array[Byte]]): DataStreamSink[T] = {
-    javaStream.writeToSocket(hostname, port, schema)
-  }
-
-  /**
-   * Adds the given sink to this DataStream. Only streams with sinks added
-   * will be executed once the StreamExecutionEnvironment.execute(...)
-   * method is called.
-   *
-   */
-  def addSink(sinkFunction: SinkFunction[T]): DataStreamSink[T] =
-    javaStream.addSink(sinkFunction)
-
-  /**
-   * Adds the given sink to this DataStream. Only streams with sinks added
-   * will be executed once the StreamExecutionEnvironment.execute(...)
-   * method is called.
-   *
-   */
-  def addSink(fun: T => Unit): DataStreamSink[T] = {
-    if (fun == null) {
-      throw new NullPointerException("Sink function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val sinkFunction = new SinkFunction[T] {
-      def invoke(in: T) = cleanFun(in)
-    }
-    this.addSink(sinkFunction)
-  }
-
-  /**
-   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-   */
-  private[flink] def clean[F <: AnyRef](f: F): F = {
-    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
deleted file mode 100644
index c259724..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
+++ /dev/null
@@ -1,313 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.functions.{FlatJoinFunction, JoinFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.streaming.api.datastream.{JoinedStreams => JavaJoinedStreams, CoGroupedStreams => JavaCoGroupedStreams}
-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.util.Collector
-
-import scala.reflect.ClassTag
-
-/**
- * `JoinedStreams` represents two [[DataStream]]s that have been joined.
- * A streaming join operation is evaluated over elements in a window.
- *
- * To finalize the join operation you also need to specify a [[KeySelector]] for
- * both the first and second input and a [[WindowAssigner]]
- *
- * Note: Right now, the groups are being built in memory so you need to ensure that they don't
- * get too big. Otherwise the JVM might crash.
- *
- * Example:
- *
- * {{{
- * val one: DataStream[(String, Int)]  = ...
- * val two: DataStream[(String, Int)] = ...
- *
- * val result = one.join(two)
- *     .where {t => ... }
- *     .equal {t => ... }
- *     .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
- *     .apply(new MyJoinFunction())
- * } }}}
- */
-object JoinedStreams {
-
-  /**
-   * A join operation that does not yet have its [[KeySelector]]s defined.
-   *
-   * @tparam T1 Type of the elements from the first input
-   * @tparam T2 Type of the elements from the second input
-   */
-  class Unspecified[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) {
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the first input.
-     */
-    def where[KEY: TypeInformation](keySelector: T1 => KEY): WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val keyType = implicitly[TypeInformation[KEY]]
-      val javaSelector = new KeySelector[T1, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T1) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = keyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, javaSelector, null, keyType)
-    }
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the second input.
-     */
-    def equalTo[KEY: TypeInformation](keySelector: T2 => KEY): WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val keyType = implicitly[TypeInformation[KEY]]
-      val javaSelector = new KeySelector[T2, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T2) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = keyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, null, javaSelector, keyType)
-    }
-
-    /**
-     * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-     * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-     */
-    private[flink] def clean[F <: AnyRef](f: F): F = {
-      new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
-    }
-  }
-
-  /**
-   * A join operation that has [[KeySelector]]s defined for either both or
-   * one input.
-   *
-   * You need to specify a [[KeySelector]] for both inputs using [[where()]] and [[equalTo()]]
-   * before you can proceeed with specifying a [[WindowAssigner]] using [[window()]].
-   *
-   * @tparam T1 Type of the elements from the first input
-   * @tparam T2 Type of the elements from the second input
-   * @tparam KEY Type of the key. This must be the same for both inputs
-   */
-  class WithKey[T1, T2, KEY](
-      input1: DataStream[T1],
-      input2: DataStream[T2],
-      keySelector1: KeySelector[T1, KEY],
-      keySelector2: KeySelector[T2, KEY],
-      keyType: TypeInformation[KEY]) {
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the first input.
-     */
-    def where(keySelector: T1 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val localKeyType = keyType
-      val javaSelector = new KeySelector[T1, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T1) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = localKeyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, javaSelector, keySelector2, localKeyType)
-    }
-
-    /**
-     * Specifies a [[KeySelector]] for elements from the second input.
-     */
-    def equalTo(keySelector: T2 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
-      val cleanFun = clean(keySelector)
-      val localKeyType = keyType
-      val javaSelector = new KeySelector[T2, KEY] with ResultTypeQueryable[KEY] {
-        def getKey(in: T2) = cleanFun(in)
-        override def getProducedType: TypeInformation[KEY] = localKeyType
-      }
-      new WithKey[T1, T2, KEY](input1, input2, keySelector1, javaSelector, localKeyType)
-    }
-
-    /**
-     * Specifies the window on which the join operation works.
-     */
-    def window[W <: Window](
-        assigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W])
-        : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
-      if (keySelector1 == null || keySelector2 == null) {
-        throw new UnsupportedOperationException("You first need to specify KeySelectors for both" +
-          "inputs using where() and equalTo().")
-      }
-      new JoinedStreams.WithWindow[T1, T2, KEY, W](
-        input1,
-        input2,
-        keySelector1,
-        keySelector2,
-        clean(assigner),
-        null,
-        null)
-    }
-
-    /**
-     * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-     * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-     */
-    private[flink] def clean[F <: AnyRef](f: F): F = {
-      new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
-    }
-  }
-
-  /**
-   * A join operation that has [[KeySelector]]s defined for both inputs as
-   * well as a [[WindowAssigner]].
-   *
-   * @tparam T1 Type of the elements from the first input
-   * @tparam T2 Type of the elements from the second input
-   * @tparam KEY Type of the key. This must be the same for both inputs
-   * @tparam W Type of { @link Window} on which the join operation works.
-   */
-  class WithWindow[T1, T2, KEY, W <: Window](
-      input1: DataStream[T1],
-      input2: DataStream[T2],
-      keySelector1: KeySelector[T1, KEY],
-      keySelector2: KeySelector[T2, KEY],
-      windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W],
-      trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W],
-      evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) {
-
-
-    /**
-     * Sets the [[Trigger]] that should be used to trigger window emission.
-     */
-    def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
-    : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
-      new WithWindow[T1, T2, KEY, W](
-        input1,
-        input2,
-        keySelector1,
-        keySelector2,
-        windowAssigner,
-        newTrigger,
-        evictor)
-    }
-
-    /**
-     * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
-     *
-     * Note: When using an evictor window performance will degrade significantly, since
-     * pre-aggregation of window results cannot be used.
-     */
-    def evictor(newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
-    : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
-      new WithWindow[T1, T2, KEY, W](
-        input1,
-        input2,
-        keySelector1,
-        keySelector2,
-        windowAssigner,
-        trigger,
-        newEvictor)
-    }
-
-    /**
-     * Completes the join operation with the user function that is executed
-     * for windowed groups.
-     */
-    def apply[O: TypeInformation: ClassTag](fun: (T1, T2) => O): DataStream[O] = {
-      require(fun != null, "Join function must not be null.")
-
-      val joiner = new FlatJoinFunction[T1, T2, O] {
-        val cleanFun = clean(fun)
-        def join(left: T1, right: T2, out: Collector[O]) = {
-          out.collect(cleanFun(left, right))
-        }
-      }
-      apply(joiner)
-    }
-
-    /**
-     * Completes the join operation with the user function that is executed
-     * for windowed groups.
-     */
-
-    def apply[O: TypeInformation: ClassTag](fun: (T1, T2, Collector[O]) => Unit): DataStream[O] = {
-      require(fun != null, "Join function must not be null.")
-
-      val joiner = new FlatJoinFunction[T1, T2, O] {
-        val cleanFun = clean(fun)
-        def join(left: T1, right: T2, out: Collector[O]) = {
-          cleanFun(left, right, out)
-        }
-      }
-      apply(joiner)
-    }
-
-    /**
-     * Completes the join operation with the user function that is executed
-     * for windowed groups.
-     */
-    def apply[T: TypeInformation](function: JoinFunction[T1, T2, T]): DataStream[T] = {
-
-      val join = new JavaJoinedStreams[T1, T2](input1.getJavaStream, input2.getJavaStream)
-
-      join
-        .where(keySelector1)
-        .equalTo(keySelector2)
-        .window(windowAssigner)
-        .trigger(trigger)
-        .evictor(evictor)
-        .apply(clean(function), implicitly[TypeInformation[T]])
-    }
-
-    /**
-     * Completes the join operation with the user function that is executed
-     * for windowed groups.
-     */
-    def apply[T: TypeInformation](function: FlatJoinFunction[T1, T2, T]): DataStream[T] = {
-
-      val join = new JavaJoinedStreams[T1, T2](input1.getJavaStream, input2.getJavaStream)
-
-      join
-        .where(keySelector1)
-        .equalTo(keySelector2)
-        .window(windowAssigner)
-        .trigger(trigger)
-        .evictor(evictor)
-        .apply(clean(function), implicitly[TypeInformation[T]])
-    }
-
-    /**
-     * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-     * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-     */
-    private[flink] def clean[F <: AnyRef](f: F): F = {
-      new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
-    }
-  }
-
-
-  /**
-   * Creates a new join operation from the two given inputs.
-   */
-  def createJoin[T1, T2](input1: DataStream[T1], input2: DataStream[T2])
-      : JoinedStreams.Unspecified[T1, T2] = {
-    new JoinedStreams.Unspecified[T1, T2](input1, input2)
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
deleted file mode 100644
index 9f5c069..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
+++ /dev/null
@@ -1,370 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.functions._
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, KeyedStream => KeyedJavaStream, WindowedStream => WindowedJavaStream}
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
-import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
-import org.apache.flink.streaming.api.operators.StreamGroupedReduce
-import org.apache.flink.streaming.api.scala.function.StatefulFunction
-import org.apache.flink.streaming.api.windowing.assigners._
-import org.apache.flink.streaming.api.windowing.time.AbstractTime
-import org.apache.flink.streaming.api.windowing.windows.{GlobalWindow, TimeWindow, Window}
-import org.apache.flink.util.Collector
-
-import scala.reflect.ClassTag
-
-
-class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T](javaStream) {
-
-  // ------------------------------------------------------------------------
-  //  Properties
-  // ------------------------------------------------------------------------
-
-  /**
-   * Gets the type of the key by which this stream is keyed.
-   */
-  def getKeyType = javaStream.getKeyType()
-  
-  // ------------------------------------------------------------------------
-  //  Windowing
-  // ------------------------------------------------------------------------
-
-  /**
-   * Windows this [[KeyedStream]] into tumbling time windows.
-   *
-   * This is a shortcut for either `.window(TumblingTimeWindows.of(size))` or
-   * `.window(TumblingProcessingTimeWindows.of(size))` depending on the time characteristic
-   * set using
-   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic()]]
-   *
-   * @param size The size of the window.
-   */
-  def timeWindow(size: AbstractTime): WindowedStream[T, K, TimeWindow] = {
-    val assigner = TumblingTimeWindows.of(size).asInstanceOf[WindowAssigner[T, TimeWindow]]
-    window(assigner)
-  }
-
-  /**
-   * Windows this [[KeyedStream]] into sliding count windows.
-   *
-   * @param size The size of the windows in number of elements.
-   * @param slide The slide interval in number of elements.
-   */
-  def countWindow(size: Long, slide: Long): WindowedStream[T, K, GlobalWindow] = {
-    new WindowedStream(javaStream.countWindow(size, slide))
-  }
-
-  /**
-   * Windows this [[KeyedStream]] into tumbling count windows.
-   *
-   * @param size The size of the windows in number of elements.
-   */
-  def countWindow(size: Long): WindowedStream[T, K, GlobalWindow] = {
-    new WindowedStream(javaStream.countWindow(size))
-  }
-
-  /**
-   * Windows this [[KeyedStream]] into sliding time windows.
-   *
-   * This is a shortcut for either `.window(SlidingTimeWindows.of(size))` or
-   * `.window(SlidingProcessingTimeWindows.of(size))` depending on the time characteristic
-   * set using
-   * [[StreamExecutionEnvironment.setStreamTimeCharacteristic()]]
-   *
-   * @param size The size of the window.
-   */
-  def timeWindow(size: AbstractTime, slide: AbstractTime): WindowedStream[T, K, TimeWindow] = {
-    val assigner = SlidingTimeWindows.of(size, slide).asInstanceOf[WindowAssigner[T, TimeWindow]]
-    window(assigner)
-  }
-
-  /**
-   * Windows this data stream to a [[WindowedStream]], which evaluates windows
-   * over a key grouped stream. Elements are put into windows by a [[WindowAssigner]]. The
-   * grouping of elements is done both by key and by window.
-   *
-   * A [[org.apache.flink.streaming.api.windowing.triggers.Trigger]] can be defined to specify
-   * when windows are evaluated. However, `WindowAssigner` have a default `Trigger`
-   * that is used if a `Trigger` is not specified.
-   *
-   * @param assigner The `WindowAssigner` that assigns elements to windows.
-   * @return The trigger windows data stream.
-   */
-  def window[W <: Window](assigner: WindowAssigner[_ >: T, W]): WindowedStream[T, K, W] = {
-    new WindowedStream(new WindowedJavaStream[T, K, W](javaStream, assigner))
-  }
-
-  // ------------------------------------------------------------------------
-  //  Non-Windowed aggregation operations
-  // ------------------------------------------------------------------------
-
-  /**
-   * Creates a new [[DataStream]] by reducing the elements of this DataStream
-   * using an associative reduce function. An independent aggregate is kept per key.
-   */
-  def reduce(reducer: ReduceFunction[T]): DataStream[T] = {
-    if (reducer == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
- 
-    javaStream.reduce(reducer)
-  }
-
-  /**
-   * Creates a new [[DataStream]] by reducing the elements of this DataStream
-   * using an associative reduce function. An independent aggregate is kept per key.
-   */
-  def reduce(fun: (T, T) => T): DataStream[T] = {
-    if (fun == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val reducer = new ReduceFunction[T] {
-      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
-    }
-    reduce(reducer)
-  }
-
-  /**
-   * Creates a new [[DataStream]] by folding the elements of this DataStream
-   * using an associative fold function and an initial value. An independent 
-   * aggregate is kept per key.
-   */
-  def fold[R: TypeInformation: ClassTag](initialValue: R, folder: FoldFunction[T,R]): 
-  DataStream[R] = {
-    if (folder == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    
-    val outType : TypeInformation[R] = implicitly[TypeInformation[R]]
-    
-    javaStream.fold(initialValue, folder).
-      returns(outType).asInstanceOf[JavaStream[R]]
-  }
-
-  /**
-   * Creates a new [[DataStream]] by folding the elements of this DataStream
-   * using an associative fold function and an initial value. An independent 
-   * aggregate is kept per key.
-   */
-  def fold[R: TypeInformation: ClassTag](initialValue: R, fun: (R,T) => R): DataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    val cleanFun = clean(fun)
-    val folder = new FoldFunction[T,R] {
-      def fold(acc: R, v: T) = {
-        cleanFun(acc, v)
-      }
-    }
-    fold(initialValue, folder)
-  }
-  
-  /**
-   * Applies an aggregation that that gives the current maximum of the data stream at
-   * the given position by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
-  
-  /**
-   * Applies an aggregation that that gives the current maximum of the data stream at
-   * the given field by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
-  
-  /**
-   * Applies an aggregation that that gives the current minimum of the data stream at
-   * the given position by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
-  
-  /**
-   * Applies an aggregation that that gives the current minimum of the data stream at
-   * the given field by the given key. An independent aggregate is kept per key.
-   *
-   */
-  def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
-
-  /**
-   * Applies an aggregation that sums the data stream at the given position by the given 
-   * key. An independent aggregate is kept per key.
-   *
-   */
-  def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
-  
-  /**
-   * Applies an aggregation that sums the data stream at the given field by the given 
-   * key. An independent aggregate is kept per key.
-   *
-   */
-  def sum(field: String): DataStream[T] =  aggregate(AggregationType.SUM, field)
-
-  /**
-   * Applies an aggregation that that gives the current minimum element of the data stream by
-   * the given position by the given key. An independent aggregate is kept per key. 
-   * When equality, the first element is returned with the minimal value.
-   *
-   */
-  def minBy(position: Int): DataStream[T] = aggregate(AggregationType
-    .MINBY, position)
-    
-   /**
-   * Applies an aggregation that that gives the current minimum element of the data stream by
-   * the given field by the given key. An independent aggregate is kept per key.
-   * When equality, the first element is returned with the minimal value.
-   *
-   */
-  def minBy(field: String): DataStream[T] = aggregate(AggregationType
-    .MINBY, field )
-
-   /**
-   * Applies an aggregation that that gives the current maximum element of the data stream by
-   * the given position by the given key. An independent aggregate is kept per key. 
-   * When equality, the first element is returned with the maximal value.
-   *
-   */
-  def maxBy(position: Int): DataStream[T] =
-    aggregate(AggregationType.MAXBY, position)
-    
-   /**
-   * Applies an aggregation that that gives the current maximum element of the data stream by
-   * the given field by the given key. An independent aggregate is kept per key. 
-   * When equality, the first element is returned with the maximal value.
-   *
-   */
-  def maxBy(field: String): DataStream[T] =
-    aggregate(AggregationType.MAXBY, field)
-    
-  private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
-    val position = fieldNames2Indices(javaStream.getType(), Array(field))(0)
-    aggregate(aggregationType, position)
-  }
-
-  private def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
-
-    val reducer = aggregationType match {
-      case AggregationType.SUM =>
-        new SumAggregator(position, javaStream.getType, javaStream.getExecutionConfig)
-      case _ =>
-        new ComparableAggregator(position, javaStream.getType, aggregationType, true,
-          javaStream.getExecutionConfig)
-    }
-
-    val invokable =  new StreamGroupedReduce[T](reducer,
-      getType().createSerializer(getExecutionConfig))
-     
-    new DataStream[T](javaStream.transform("aggregation", javaStream.getType(),invokable))
-      .asInstanceOf[DataStream[T]]
-  }
-
-  // ------------------------------------------------------------------------
-  //  functions with state
-  // ------------------------------------------------------------------------
-  
-  /**
-   * Creates a new DataStream that contains only the elements satisfying the given stateful filter 
-   * predicate. To use state partitioning, a key must be defined using .keyBy(..), in which case
-   * an independent state will be kept per key.
-   *
-   * Note that the user state object needs to be serializable.
-   */
-  def filterWithState[S : TypeInformation](
-        fun: (T, Option[S]) => (Boolean, Option[S])): DataStream[T] = {
-    if (fun == null) {
-      throw new NullPointerException("Filter function must not be null.")
-    }
-
-    val cleanFun = clean(fun)
-    val stateTypeInfo: TypeInformation[S] = implicitly[TypeInformation[S]]
-
-    val filterFun = new RichFilterFunction[T] with StatefulFunction[T, Boolean, S] {
-
-      override val stateType: TypeInformation[S] = stateTypeInfo
-
-      override def filter(in: T): Boolean = {
-        applyWithState(in, cleanFun)
-      }
-    }
-
-    filter(filterFun)
-  }
-
-  /**
-   * Creates a new DataStream by applying the given stateful function to every element of this 
-   * DataStream. To use state partitioning, a key must be defined using .keyBy(..), in which 
-   * case an independent state will be kept per key.
-   *
-   * Note that the user state object needs to be serializable.
-   */
-  def mapWithState[R: TypeInformation: ClassTag, S: TypeInformation](
-        fun: (T, Option[S]) => (R, Option[S])): DataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("Map function must not be null.")
-    }
-
-    val cleanFun = clean(fun)
-    val stateTypeInfo: TypeInformation[S] = implicitly[TypeInformation[S]]
-    
-    val mapper = new RichMapFunction[T, R] with StatefulFunction[T, R, S] {
-
-      override val stateType: TypeInformation[S] = stateTypeInfo
-      
-      override def map(in: T): R = {
-        applyWithState(in, cleanFun)
-      }
-    }
-
-    map(mapper)
-  }
-  
-  /**
-   * Creates a new DataStream by applying the given stateful function to every element and 
-   * flattening the results. To use state partitioning, a key must be defined using .keyBy(..), 
-   * in which case an independent state will be kept per key.
-   *
-   * Note that the user state object needs to be serializable.
-   */
-  def flatMapWithState[R: TypeInformation: ClassTag, S: TypeInformation](
-        fun: (T, Option[S]) => (TraversableOnce[R], Option[S])): DataStream[R] = {
-    if (fun == null) {
-      throw new NullPointerException("Flatmap function must not be null.")
-    }
-
-    val cleanFun = clean(fun)
-    val stateTypeInfo: TypeInformation[S] = implicitly[TypeInformation[S]]
-    
-    val flatMapper = new RichFlatMapFunction[T, R] with StatefulFunction[T,TraversableOnce[R],S]{
-
-      override val stateType: TypeInformation[S] = stateTypeInfo
-      
-      override def flatMap(in: T, out: Collector[R]): Unit = {
-        applyWithState(in, cleanFun) foreach out.collect
-      }
-    }
-
-    flatMap(flatMapper)
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala
deleted file mode 100644
index deea6f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/SplitStream.scala
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.streaming.api.datastream.{ SplitStream => SplitJavaStream }
-
-/**
- * The SplitStream represents an operator that has been split using an
- * {@link OutputSelector}. Named outputs can be selected using the
- * {@link #select} function. To apply a transformation on the whole output simply call
- * the appropriate method on this stream.
- *
- */
-class SplitStream[T](javaStream: SplitJavaStream[T]) extends DataStream[T](javaStream){
-
-  /**
-   *  Sets the output names for which the next operator will receive values.
-   */
-  def select(outputNames: String*): DataStream[T] = javaStream.select(outputNames: _*)
-  
-}


[14/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index e953696..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ /dev/null
@@ -1,657 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import java.util.Objects
-import java.util.Objects._
-
-import com.esotericsoftware.kryo.Serializer
-import org.apache.flink.api.common.io.{FileInputFormat, InputFormat}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer
-import org.apache.flink.api.scala.ClosureCleaner
-import org.apache.flink.runtime.state.StateBackend
-import org.apache.flink.streaming.api.{TimeCharacteristic, CheckpointingMode}
-import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaEnv}
-import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
-import org.apache.flink.streaming.api.functions.source.SourceFunction
-import org.apache.flink.types.StringValue
-import org.apache.flink.util.SplittableIterator
-
-import scala.collection.JavaConverters._
-import scala.reflect.ClassTag
-
-import _root_.scala.language.implicitConversions
-
-class StreamExecutionEnvironment(javaEnv: JavaEnv) {
-
-  /**
-   * Gets the config object.
-   */
-  def getConfig = javaEnv.getConfig
-
-  /**
-   * 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)
-  }
-
-  /**
-   * Returns the default parallelism for this execution environment. Note that this
-   * value can be overridden by individual operations using [[DataStream#setParallelism(int)]]
-   */
-  def getParallelism = javaEnv.getParallelism
-
-  /**
-   * Sets the maximum time frequency (milliseconds) for the flushing of the
-   * output buffers. By default the output buffers flush frequently to provide
-   * low latency and to aid smooth developer experience. Setting the parameter
-   * can result in three logical modes:
-   *
-   * <ul>
-   *   <li>A positive integer triggers flushing periodically by that integer</li>
-   *   <li>0 triggers flushing after every record thus minimizing latency</li>
-   *   <li>-1 triggers flushing only when the output buffer is full thus maximizing throughput</li>
-   * </ul>
-   */
-  def setBufferTimeout(timeoutMillis: Long): StreamExecutionEnvironment = {
-    javaEnv.setBufferTimeout(timeoutMillis)
-    this
-  }
-
-  /**
-   * Gets the default buffer timeout set for this environment
-   */
-  def getBufferTimeout = javaEnv.getBufferTimeout
-
-  /**
-   * Disables operator chaining for streaming operators. Operator chaining
-   * allows non-shuffle operations to be co-located in the same thread fully
-   * avoiding serialization and de-serialization.
-   *
-   */
-  def disableOperatorChaining(): StreamExecutionEnvironment = {
-    javaEnv.disableOperatorChaining()
-    this
-  }
-
-  // ------------------------------------------------------------------------
-  //  Checkpointing Settings
-  // ------------------------------------------------------------------------
-  /**
-   * Enables checkpointing for the streaming job. The distributed state of the streaming
-   * dataflow will be periodically snapshotted. In case of a failure, the streaming
-   * dataflow will be restarted from the latest completed checkpoint.
-   *
-   * The job draws checkpoints periodically, in the given interval. The state will be
-   * stored in the configured state backend.
-   *
-   * NOTE: Checkpointing iterative streaming dataflows in not properly supported at
-   * the moment. If the "force" parameter is set to true, the system will execute the
-   * job nonetheless.
-   *
-   * @param interval
-   *     Time interval between state checkpoints in millis.
-   * @param mode
-   *     The checkpointing mode, selecting between "exactly once" and "at least once" guarantees.
-   * @param force
-   *           If true checkpointing will be enabled for iterative jobs as well.
-   */
-  @deprecated
-  def enableCheckpointing(interval : Long,
-                          mode: CheckpointingMode,
-                          force: Boolean) : StreamExecutionEnvironment = {
-    javaEnv.enableCheckpointing(interval, mode, force)
-    this
-  }
-
-  /**
-   * Enables checkpointing for the streaming job. The distributed state of the streaming
-   * dataflow will be periodically snapshotted. In case of a failure, the streaming
-   * dataflow will be restarted from the latest completed checkpoint.
-   *
-   * The job draws checkpoints periodically, in the given interval. The system uses the
-   * given [[CheckpointingMode]] for the checkpointing ("exactly once" vs "at least once").
-   * The state will be stored in the configured state backend.
-   *
-   * NOTE: Checkpointing iterative streaming dataflows in not properly supported at
-   * the moment. For that reason, iterative jobs will not be started if used
-   * with enabled checkpointing. To override this mechanism, use the 
-   * [[enableCheckpointing(long, CheckpointingMode, boolean)]] method.
-   *
-   * @param interval 
-   *     Time interval between state checkpoints in milliseconds.
-   * @param mode 
-   *     The checkpointing mode, selecting between "exactly once" and "at least once" guarantees.
-   */
-  def enableCheckpointing(interval : Long,
-                          mode: CheckpointingMode) : StreamExecutionEnvironment = {
-    javaEnv.enableCheckpointing(interval, mode)
-    this
-  }
-
-  /**
-   * Enables checkpointing for the streaming job. The distributed state of the streaming
-   * dataflow will be periodically snapshotted. In case of a failure, the streaming
-   * dataflow will be restarted from the latest completed checkpoint.
-   *
-   * The job draws checkpoints periodically, in the given interval. The program will use
-   * [[CheckpointingMode.EXACTLY_ONCE]] mode. The state will be stored in the
-   * configured state backend.
-   *
-   * NOTE: Checkpointing iterative streaming dataflows in not properly supported at
-   * the moment. For that reason, iterative jobs will not be started if used
-   * with enabled checkpointing. To override this mechanism, use the 
-   * [[enableCheckpointing(long, CheckpointingMode, boolean)]] method.
-   *
-   * @param interval 
-   *           Time interval between state checkpoints in milliseconds.
-   */
-  def enableCheckpointing(interval : Long) : StreamExecutionEnvironment = {
-    enableCheckpointing(interval, CheckpointingMode.EXACTLY_ONCE)
-  }
-
-  /**
-   * Method for enabling fault-tolerance. Activates monitoring and backup of streaming
-   * operator states. Time interval between state checkpoints is specified in in millis.
-   *
-   * Setting this option assumes that the job is used in production and thus if not stated
-   * explicitly otherwise with calling with the
-   * [[setNumberOfExecutionRetries(int)]] method in case of
-   * failure the job will be resubmitted to the cluster indefinitely.
-   */
-  def enableCheckpointing() : StreamExecutionEnvironment = {
-    javaEnv.enableCheckpointing()
-    this
-  }
-  
-  def getCheckpointingMode = javaEnv.getCheckpointingMode()
-
-  /**
-   * Sets the state backend that describes how to store and checkpoint operator state.
-   * It defines in what form the key/value state, accessible from operations on
-   * [[KeyedStream]] is maintained (heap, managed memory, externally), and where state
-   * snapshots/checkpoints are stored, both for the key/value state, and for checkpointed
-   * functions (implementing the interface 
-   * [[org.apache.flink.streaming.api.checkpoint.Checkpointed]].
-   *
-   * <p>The [[org.apache.flink.streaming.api.state.memory.MemoryStateBackend]] for example
-   * maintains the state in heap memory, as objects. It is lightweight without extra 
-   * dependencies, but can checkpoint only small states (some counters).
-   *
-   * <p>In contrast, the [[org.apache.flink.streaming.api.state.filesystem.FsStateBackend]]
-   * stores checkpoints of the state (also maintained as heap objects) in files. When using
-   * a replicated file system (like HDFS, S3, MapR FS, Tachyon, etc) this will guarantee
-   * that state is not lost upon failures of individual nodes and that the entire streaming
-   * program can be executed highly available and strongly consistent (assuming that Flink
-   * is run in high-availability mode).
-   */
-  def setStateBackend(backend: StateBackend[_]): StreamExecutionEnvironment = {
-    javaEnv.setStateBackend(backend)
-    this
-  }
-
-  /**
-   * Returns the state backend that defines how to store and checkpoint state.
-   */
-  def getStateBackend: StateBackend[_] = javaEnv.getStateBackend()
-  
-  /**
-   * Sets the number of times that failed tasks are re-executed. A value of zero
-   * effectively disables fault tolerance. A value of "-1" indicates that the system
-   * default value (as defined in the configuration) should be used.
-   */
-  def setNumberOfExecutionRetries(numRetries: Int): Unit = {
-    javaEnv.setNumberOfExecutionRetries(numRetries)
-  }
-
-  /**
-   * Gets the number of times the system will try to re-execute failed tasks. A value
-   * of "-1" indicates that the system default value (as defined in the configuration)
-   * should be used.
-   */
-  def getNumberOfExecutionRetries = javaEnv.getNumberOfExecutionRetries
-
-  /**
-   * Sets the delay that failed tasks are re-executed. A value of
-   * zero effectively disables fault tolerance. A value of "-1"
-   * indicates that the system default value (as defined in the configuration)
-   * should be used.
-   */
-  def setExecutionRetryDelay(executionRetryDelay: Long): Unit = {
-    javaEnv.setExecutionRetryDelay(executionRetryDelay)
-  }
-
-  /**
-   * Gets the delay time in milliseconds the system will wait to re-execute failed tasks.
-   * A value of "-1" indicates that the system default value (as defined
-   * in the configuration) should be used.
-   */
-  def getExecutionRetryDelay = javaEnv.getExecutionRetryDelay
-
-  // --------------------------------------------------------------------------------------------
-  // Registry for types and serializers
-  // --------------------------------------------------------------------------------------------
-  /**
-   * Adds a new Kryo default serializer to the Runtime.
-   * <p/>
-   * Note that the serializer instance must be serializable (as defined by
-   * java.io.Serializable), because it may be distributed to the worker nodes
-   * by java serialization.
-   *
-   * @param type
-   * The class of the types serialized with the given serializer.
-   * @param serializer
-   * The serializer to use.
-   */
-  def addDefaultKryoSerializer[T <: Serializer[_] with Serializable](
-      `type`: Class[_],
-      serializer: T)
-    : Unit = {
-    javaEnv.addDefaultKryoSerializer(`type`, serializer)
-  }
-
-  /**
-   * Adds a new Kryo default serializer to the Runtime.
-   *
-   * @param type
-   * The class of the types serialized with the given serializer.
-   * @param serializerClass
-   * The class of the serializer to use.
-   */
-  def addDefaultKryoSerializer(`type`: Class[_], serializerClass: Class[_ <: Serializer[_]]) {
-    javaEnv.addDefaultKryoSerializer(`type`, serializerClass)
-  }
-
-  /**
-   * Registers the given type with the serializer at the [[KryoSerializer]].
-   *
-   * Note that the serializer instance must be serializable (as defined by java.io.Serializable),
-   * because it may be distributed to the worker nodes by java serialization.
-   */
-  def registerTypeWithKryoSerializer[T <: Serializer[_] with Serializable](
-      clazz: Class[_],
-      serializer: T)
-    : Unit = {
-    javaEnv.registerTypeWithKryoSerializer(clazz, serializer)
-  }
-
-  /**
-   * Registers the given type with the serializer at the [[KryoSerializer]].
-   */
-  def registerTypeWithKryoSerializer(clazz: Class[_], serializer: Class[_ <: Serializer[_]]) {
-    javaEnv.registerTypeWithKryoSerializer(clazz, serializer)
-  }
-
-  /**
-   * Registers the given type with the serialization stack. If the type is eventually
-   * serialized as a POJO, then the type is registered with the POJO serializer. If the
-   * type ends up being serialized with Kryo, then it will be registered at Kryo to make
-   * sure that only tags are written.
-   *
-   */
-  def registerType(typeClass: Class[_]) {
-    javaEnv.registerType(typeClass)
-  }
-
-  // --------------------------------------------------------------------------------------------
-  //  Time characteristic
-  // --------------------------------------------------------------------------------------------
-  /**
-   * Sets the time characteristic for all streams create from this environment, e.g., processing
-   * time, event time, or ingestion time.
-   *
-   * If you set the characteristic to IngestionTime of EventTime this will set a default
-   * watermark update interval of 200 ms. If this is not applicable for your application
-   * you should change it using
-   * [[org.apache.flink.api.common.ExecutionConfig#setAutoWatermarkInterval(long)]]
-   *
-   * @param characteristic The time characteristic.
-   */
-  def setStreamTimeCharacteristic(characteristic: TimeCharacteristic) : Unit = {
-    javaEnv.setStreamTimeCharacteristic(characteristic)
-  }
-
-  /**
-   * Gets the time characteristic/
-   *
-   * @see #setStreamTimeCharacteristic
-   *
-   * @return The time characteristic.
-   */
-  def getStreamTimeCharacteristic = javaEnv.getStreamTimeCharacteristic()
-
-  // --------------------------------------------------------------------------------------------
-  // Data stream creations
-  // --------------------------------------------------------------------------------------------
-
-  /**
-   * Creates a new DataStream that contains a sequence of numbers. This source is a parallel source.
-   * If you manually set the parallelism to `1` the emitted elements are in order.
-   */
-  def generateSequence(from: Long, to: Long): DataStream[Long] = {
-    new DataStream[java.lang.Long](javaEnv.generateSequence(from, to))
-      .asInstanceOf[DataStream[Long]]
-  }
-
-  /**
-   * Creates a DataStream that contains the given elements. The elements must all be of the
-   * same type.
-   *
-   * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a parallelism of one.
-   */
-  def fromElements[T: ClassTag: TypeInformation](data: T*): DataStream[T] = {
-    val typeInfo = implicitly[TypeInformation[T]]
-    fromCollection(data)(implicitly[ClassTag[T]], typeInfo)
-  }
-
-  /**
-   * Creates a DataStream from the given non-empty [[Seq]]. The elements need to be serializable
-   * because the framework may move the elements into the cluster if needed.
-   *
-   * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a parallelism of one.
-   */
-  def fromCollection[T: ClassTag: TypeInformation](data: Seq[T]): DataStream[T] = {
-    require(data != null, "Data must not be null.")
-    val typeInfo = implicitly[TypeInformation[T]]
-
-    javaEnv.fromCollection(scala.collection.JavaConversions.asJavaCollection(data), typeInfo)
-  }
-
-  /**
-   * Creates a DataStream from the given [[Iterator]].
-   *
-   * Note that this operation will result in a non-parallel data source, i.e. a data source with
-   * a parallelism of one.
-   */
-  def fromCollection[T: ClassTag : TypeInformation] (data: Iterator[T]): DataStream[T] = {
-    val typeInfo = implicitly[TypeInformation[T]]
-    javaEnv.fromCollection(data.asJava, typeInfo)
-  }
-
-  /**
-   * Creates a DataStream from the given [[SplittableIterator]].
-   */
-  def fromParallelCollection[T: ClassTag : TypeInformation] (data: SplittableIterator[T]):
-  DataStream[T] = {
-    val typeInfo = implicitly[TypeInformation[T]]
-    javaEnv.fromParallelCollection(data, typeInfo)
-  }
-
-  /**
-   * Creates a DataStream that represents the Strings produced by reading the
-   * given file line wise. The file will be read with the system's default
-   * character set.
-   *
-   */
-  def readTextFile(filePath: String): DataStream[String] =
-    javaEnv.readTextFile(filePath)
-
-  /**
-   * Creates a data stream that represents the Strings produced by reading the given file
-   * line wise. The character set with the given name will be used to read the files.
-   */
-  def readTextFile(filePath: String, charsetName: String): DataStream[String] =
-    javaEnv.readTextFile(filePath, charsetName)
-
-  /**
-   * Creates a data stream that represents the strings produced by reading the given file
-   * line wise. This method is similar to the standard text file reader, but it produces
-   * a data stream with mutable StringValue objects, rather than Java Strings.
-   * StringValues can be used to tune implementations to be less object and garbage
-   * collection heavy. The file will be read with the system's default character set.
-   */
-  def readTextFileWithValue(filePath: String): DataStream[StringValue] =
-      javaEnv.readTextFileWithValue(filePath)
-
-  /**
-   * Creates a data stream that represents the strings produced by reading the given file
-   * line wise. This method is similar to the standard text file reader, but it produces
-   * a data stream with mutable StringValue objects, rather than Java Strings.
-   * StringValues can be used to tune implementations to be less object and garbage
-   * collection heavy. The boolean flag indicates whether to skip lines that cannot
-   * be read with the given character set.
-   */
-  def readTextFileWithValue(filePath: String, charsetName : String, skipInvalidLines : Boolean):
-    DataStream[StringValue] =
-    javaEnv.readTextFileWithValue(filePath, charsetName, skipInvalidLines)
-
-  /**
-   * Reads the given file with the given input format. The file path should be passed
-   * as a URI (e.g., "file:///some/local/file" or "hdfs://host:port/file/path").
-   */
-  def readFile[T: ClassTag : TypeInformation](inputFormat: FileInputFormat[T], filePath: String):
-    DataStream[T] =
-    javaEnv.readFile(inputFormat, filePath)
-
-  /**
-   * Creates a data stream that represents the primitive type produced by reading the given file
-   * line wise. The file path should be passed as a URI (e.g., "file:///some/local/file" or
-   * "hdfs://host:port/file/path").
-   */
-  def readFileOfPrimitives[T: ClassTag : TypeInformation](filePath: String,
-    delimiter: String = "\n", typeClass: Class[T]): DataStream[T] =
-    javaEnv.readFileOfPrimitives(filePath, delimiter, typeClass)
-
-  /**
-   * Creates a DataStream that contains the contents of file created while
-   * system watches the given path. The file will be read with the system's
-   * default character set. The user can check the monitoring interval in milliseconds,
-   * and the way file modifications are handled. By default it checks for only new files
-   * every 100 milliseconds.
-   *
-   */
-  def readFileStream(StreamPath: String, intervalMillis: Long = 100, watchType: WatchType = 
-    WatchType.ONLY_NEW_FILES): DataStream[String] =
-    javaEnv.readFileStream(StreamPath, intervalMillis, watchType)
-
-  /**
-   * Creates a new DataStream that contains the strings received infinitely
-   * from socket. Received strings are decoded by the system's default
-   * character set. The maximum retry interval is specified in seconds, in case
-   * of temporary service outage reconnection is initiated every second.
-   */
-  def socketTextStream(hostname: String, port: Int, delimiter: Char = '\n', maxRetry: Long = 0):
-    DataStream[String] =
-    javaEnv.socketTextStream(hostname, port)
-
-  /**
-   * Generic method to create an input data stream with a specific input format.
-   * Since all data streams need specific information about their types, this method needs to
-   * determine the type of the data produced by the input format. It will attempt to determine the
-   * data type by reflection, unless the input format implements the ResultTypeQueryable interface.
-   */
-  def createInput[T: ClassTag : TypeInformation](inputFormat: InputFormat[T, _]): DataStream[T] =
-    javaEnv.createInput(inputFormat)
-
-  /**
-   * Create a DataStream using a user defined source function for arbitrary
-   * source functionality. By default sources have a parallelism of 1. 
-   * To enable parallel execution, the user defined source should implement 
-   * ParallelSourceFunction or extend RichParallelSourceFunction. 
-   * In these cases the resulting source will have the parallelism of the environment. 
-   * To change this afterwards call DataStreamSource.setParallelism(int)
-   *
-   */
-  def addSource[T: ClassTag: TypeInformation](function: SourceFunction[T]): DataStream[T] = {
-    require(function != null, "Function must not be null.")
-    val cleanFun = scalaClean(function)
-    val typeInfo = implicitly[TypeInformation[T]]
-    javaEnv.addSource(cleanFun).returns(typeInfo)
-  }
-
-  /**
-   * Create a DataStream using a user defined source function for arbitrary
-   * source functionality.
-   *
-   */
-  def addSource[T: ClassTag: TypeInformation](function: SourceContext[T] => Unit): DataStream[T] = {
-    require(function != null, "Function must not be null.")
-    val sourceFunction = new SourceFunction[T] {
-      val cleanFun = scalaClean(function)
-      override def run(ctx: SourceContext[T]) {
-        cleanFun(ctx)
-      }
-      override def cancel() = {}
-    }
-    addSource(sourceFunction)
-  }
-
-  /**
-   * 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.
-   *
-   */
-  def execute() = javaEnv.execute()
-
-  /**
-   * 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
-   *
-   */
-  def execute(jobName: String) = javaEnv.execute(jobName)
-
-  /**
-   * 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.
-   *
-   */
-  def getExecutionPlan = javaEnv.getExecutionPlan
-
-  /**
-   * Getter of the [[org.apache.flink.streaming.api.graph.StreamGraph]] of the streaming job.
-   *
-   * @return The StreamGraph representing the transformations
-   */
-  def getStreamGraph = javaEnv.getStreamGraph
-
-  /**
-   * Getter of the wrapped [[org.apache.flink.streaming.api.environment.StreamExecutionEnvironment]]
-   * @return The encased ExecutionEnvironment
-   */
-  def getWrappedStreamExecutionEnvironment = javaEnv
-
-  /**
-   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]]
-   */
-  private[flink] def scalaClean[F <: AnyRef](f: F): F = {
-    if (getConfig.isClosureCleanerEnabled) {
-      ClosureCleaner.clean(f, true)
-    } else {
-      ClosureCleaner.ensureSerializable(f)
-    }
-    f
-  }
-}
-
-object StreamExecutionEnvironment {
-
-  /**
-   * Sets the default parallelism that will be used for the local execution
-   * environment created by [[createLocalEnvironment()]].
-   *
-   * @param parallelism
-   * The parallelism to use as the default local parallelism.
-   */
-  def setDefaultLocalParallelism(parallelism: Int) : Unit =
-    StreamExecutionEnvironment.setDefaultLocalParallelism(parallelism)
-
-  /**
-   * Creates an execution environment that represents the context in which the program is
-   * currently executed. If the program is invoked standalone, this method returns a local
-   * execution environment. If the program is invoked from within the command line client
-   * to be submitted to a cluster, this method returns the execution environment of this cluster.
-   */
-  def getExecutionEnvironment: StreamExecutionEnvironment = {
-    new StreamExecutionEnvironment(JavaEnv.getExecutionEnvironment)
-  }
-
-  /**
-   * Creates a local execution environment. The local execution environment will run the program in
-   * a multi-threaded fashion in the same JVM as the environment was created in. The default degree
-   * of parallelism of the local environment is the number of hardware contexts (CPU cores/threads).
-   */
-  def createLocalEnvironment(
-    parallelism: Int =  Runtime.getRuntime.availableProcessors()):
-  StreamExecutionEnvironment = {
-    new StreamExecutionEnvironment(JavaEnv.createLocalEnvironment(parallelism))
-  }
-
-  /**
-   * Creates a remote execution environment. 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 cluster's default parallelism, unless the
-   * parallelism is set explicitly via [[StreamExecutionEnvironment.setParallelism()]].
-   *
-   * @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 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.
-   */
-  def createRemoteEnvironment(host: String, port: Int, jarFiles: String*):
-  StreamExecutionEnvironment = {
-    new StreamExecutionEnvironment(JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*))
-  }
-
-  /**
-   * Creates a remote execution environment. 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.
-   *
-   * @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 parallelism The parallelism to use during the execution.
-   * @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.
-   */
-  def createRemoteEnvironment(
-    host: String,
-    port: Int,
-    parallelism: Int,
-    jarFiles: String*): StreamExecutionEnvironment = {
-    val javaEnv = JavaEnv.createRemoteEnvironment(host, port, jarFiles: _*)
-    javaEnv.setParallelism(parallelism)
-    new StreamExecutionEnvironment(javaEnv)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
deleted file mode 100644
index 93b91ff..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
+++ /dev/null
@@ -1,371 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream}
-import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
-import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction
-import org.apache.flink.streaming.api.windowing.evictors.Evictor
-import org.apache.flink.streaming.api.windowing.triggers.Trigger
-import org.apache.flink.streaming.api.windowing.windows.Window
-import org.apache.flink.util.Collector
-
-import scala.reflect.ClassTag
-
-import scala.collection.JavaConverters._
-
-/**
- * A [[WindowedStream]] represents a data stream where elements are grouped by
- * key, and for each key, the stream of elements is split into windows based on a
- * [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]. Window emission
- * is triggered based on a [[Trigger]].
- *
- * The windows are conceptually evaluated for each key individually, meaning windows can trigger at
- * different points for each key.
- *
- * If an [[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 [[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.
- *
- * Note that the [[WindowedStream]] is purely and API construct, during runtime
- * the [[WindowedStream]] will be collapsed together with the
- * [[KeyedStream]] and the operation over the window into one single operation.
- *
- * @tparam T The type of elements in the stream.
- * @tparam K The type of the key by which elements are grouped.
- * @tparam W The type of [[Window]] that the
- *           [[org.apache.flink.streaming.api.windowing.assigners.WindowAssigner]]
- *           assigns the elements to.
- */
-class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
-
-  /**
-   * Sets the [[Trigger]] that should be used to trigger window emission.
-   */
-  def trigger(trigger: Trigger[_ >: T, _ >: W]): WindowedStream[T, K, W] = {
-    javaStream.trigger(trigger)
-    this
-  }
-
-  /**
-   * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
-   *
-   * Note: When using an evictor window performance will degrade significantly, since
-   * pre-aggregation of window results cannot be used.
-   */
-  def evictor(evictor: Evictor[_ >: T, _ >: W]): WindowedStream[T, K, W] = {
-    javaStream.evictor(evictor)
-    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.
-   *
-   * 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.
-   */
-  def reduce(function: ReduceFunction[T]): DataStream[T] = {
-    javaStream.reduce(clean(function))
-  }
-
-  /**
-   * 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.
-   *
-   * 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.
-   */
-  def reduce(function: (T, T) => T): DataStream[T] = {
-    if (function == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    val cleanFun = clean(function)
-    val reducer = new ReduceFunction[T] {
-      def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
-    }
-    reduce(reducer)
-  }
-
-  /**
-   * Applies the given fold function to each 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.
-   *
-   * @param function The fold function.
-   * @return The data stream that is the result of applying the fold function to the window.
-   */
-  def fold[R: TypeInformation: ClassTag](
-      initialValue: R,
-      function: FoldFunction[T,R]): DataStream[R] = {
-    if (function == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-
-    val resultType : TypeInformation[R] = implicitly[TypeInformation[R]]
-
-    javaStream.fold(initialValue, function, resultType)
-  }
-
-  /**
-   * Applies the given fold function to each 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.
-   *
-   * @param function The fold function.
-   * @return The data stream that is the result of applying the fold function to the window.
-   */
-  def fold[R: TypeInformation: ClassTag](initialValue: R, function: (R, T) => R): DataStream[R] = {
-    if (function == null) {
-      throw new NullPointerException("Fold function must not be null.")
-    }
-    val cleanFun = clean(function)
-    val folder = new FoldFunction[T,R] {
-      def fold(acc: R, v: T) = {
-        cleanFun(acc, v)
-      }
-    }
-    fold(initialValue, folder)
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Not that this function requires that all data in the windows is buffered until the window
-   * is evaluated, as the function provides no means of pre-aggregation.
-   *
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](function: WindowFunction[T, R, K, W]): DataStream[R] = {
-    javaStream.apply(clean(function), implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Not that this function requires that all data in the windows is buffered until the window
-   * is evaluated, as the function provides no means of pre-aggregation.
-   *
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](
-      function: (K, W, Iterable[T], Collector[R]) => Unit): DataStream[R] = {
-    if (function == null) {
-      throw new NullPointerException("WindowApply function must not be null.")
-    }
-
-    val cleanedFunction = clean(function)
-    val applyFunction = new WindowFunction[T, R, K, W] {
-      def apply(key: K, window: W, elements: java.lang.Iterable[T], out: Collector[R]): Unit = {
-        cleanedFunction(key, window, elements.asScala, out)
-      }
-    }
-    javaStream.apply(applyFunction, implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-   *
-   * @param preAggregator The reduce function that is used for pre-aggregation
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](
-      preAggregator: ReduceFunction[T],
-      function: WindowFunction[T, R, K, W]): DataStream[R] = {
-    javaStream.apply(clean(preAggregator), clean(function), implicitly[TypeInformation[R]])
-  }
-
-  /**
-   * Applies the given window function to each window. The window function is called for each
-   * evaluation of the window for each key individually. The output of the window function is
-   * interpreted as a regular non-windowed stream.
-   *
-   * Arriving data is pre-aggregated using the given pre-aggregation reducer.
-   *
-   * @param preAggregator The reduce function that is used for pre-aggregation
-   * @param function The window function.
-   * @return The data stream that is the result of applying the window function to the window.
-   */
-  def apply[R: TypeInformation: ClassTag](
-      preAggregator: (T, T) => T,
-      function: (K, W, Iterable[T], Collector[R]) => Unit): DataStream[R] = {
-    if (function == null) {
-      throw new NullPointerException("Reduce function must not be null.")
-    }
-    if (function == null) {
-      throw new NullPointerException("WindowApply function must not be null.")
-    }
-
-    val cleanReducer = clean(preAggregator)
-    val reducer = new ReduceFunction[T] {
-      def reduce(v1: T, v2: T) = { cleanReducer(v1, v2) }
-    }
-
-    val cleanApply = clean(function)
-    val applyFunction = new WindowFunction[T, R, K, W] {
-      def apply(key: K, window: W, elements: java.lang.Iterable[T], out: Collector[R]): Unit = {
-        cleanApply(key, window, elements.asScala, out)
-      }
-    }
-    javaStream.apply(reducer, applyFunction, implicitly[TypeInformation[R]])
-  }
-
-  // ------------------------------------------------------------------------
-  //  Aggregations on the keyed windows
-  // ------------------------------------------------------------------------
-
-  /**
-   * Applies an aggregation that that gives the maximum of the elements in the window at
-   * the given position.
-   */
-  def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
-
-  /**
-   * Applies an aggregation that that gives the maximum of the elements in the window at
-   * the given field.
-   */
-  def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
-
-  /**
-   * Applies an aggregation that that gives the minimum of the elements in the window at
-   * the given position.
-   */
-  def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
-
-  /**
-   * Applies an aggregation that that gives the minimum of the elements in the window at
-   * the given field.
-   */
-  def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
-
-  /**
-   * Applies an aggregation that sums the elements in the window at the given position.
-   */
-  def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
-
-  /**
-   * Applies an aggregation that sums the elements in the window at the given field.
-   */
-  def sum(field: String): DataStream[T] = aggregate(AggregationType.SUM, field)
-
-  /**
-   * Applies an aggregation that that gives the maximum element of the window by
-   * the given position. When equality, returns the first.
-   */
-  def maxBy(position: Int): DataStream[T] = aggregate(AggregationType.MAXBY,
-    position)
-
-  /**
-   * Applies an aggregation that that gives the maximum element of the window by
-   * the given field. When equality, returns the first.
-   */
-  def maxBy(field: String): DataStream[T] = aggregate(AggregationType.MAXBY,
-    field)
-
-  /**
-   * Applies an aggregation that that gives the minimum element of the window by
-   * the given position. When equality, returns the first.
-   */
-  def minBy(position: Int): DataStream[T] = aggregate(AggregationType.MINBY,
-    position)
-
-  /**
-   * Applies an aggregation that that gives the minimum element of the window by
-   * the given field. When equality, returns the first.
-   */
-  def minBy(field: String): DataStream[T] = aggregate(AggregationType.MINBY,
-    field)
-
-  private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
-    val position = fieldNames2Indices(getInputType(), Array(field))(0)
-    aggregate(aggregationType, position)
-  }
-
-  def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
-
-    val jStream = javaStream.asInstanceOf[JavaWStream[Product, K, W]]
-
-    val reducer = aggregationType match {
-      case AggregationType.SUM =>
-        new SumAggregator(position, jStream.getInputType, jStream.getExecutionEnvironment.getConfig)
-
-      case _ =>
-        new ComparableAggregator(
-          position,
-          jStream.getInputType,
-          aggregationType,
-          true,
-          jStream.getExecutionEnvironment.getConfig)
-    }
-
-    new DataStream[Product](jStream.reduce(reducer)).asInstanceOf[DataStream[T]]
-  }
-
-  // ------------------------------------------------------------------------
-  //  Utilities
-  // ------------------------------------------------------------------------
-
-  /**
-   * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
-   * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
-   */
-  private[flink] def clean[F <: AnyRef](f: F): F = {
-    new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
-  }
-
-  /**
-   * Gets the output type.
-   */
-  private def getInputType(): TypeInformation[T] = javaStream.getInputType
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
deleted file mode 100644
index d66cfdb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala.function
-
-import org.apache.flink.api.common.functions.RichFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.api.common.state.OperatorState
-
-/**
- * Trait implementing the functionality necessary to apply stateful functions in 
- * RichFunctions without exposing the OperatorStates to the user. The user should
- * call the applyWithState method in his own RichFunction implementation.
- */
-trait StatefulFunction[I, O, S] extends RichFunction {
-  
-  var state: OperatorState[S] = _
-  val stateType: TypeInformation[S]
-
-  def applyWithState(in: I, fun: (I, Option[S]) => (O, Option[S])): O = {
-    val (o, s: Option[S]) = fun(in, Option(state.value()))
-    s match {
-      case Some(v) => state.update(v)
-      case None => state.update(null.asInstanceOf[S])
-    }
-    o
-  }
-
-  override def open(c: Configuration) = {
-    state = getRuntimeContext().getKeyValueState[S]("state", stateType, null.asInstanceOf[S])
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
deleted file mode 100644
index e668064..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/package.scala
+++ /dev/null
@@ -1,76 +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
-
-import _root_.scala.reflect.ClassTag
-import language.experimental.macros
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.{createTuple2TypeInformation => apiTupleCreator}
-import org.apache.flink.api.scala.typeutils.{CaseClassTypeInfo, TypeUtils}
-import org.apache.flink.streaming.api.datastream.{ DataStream => JavaStream }
-import org.apache.flink.streaming.api.datastream.{ SplitStream => SplitJavaStream }
-import org.apache.flink.streaming.api.datastream.{ ConnectedStreams => ConnectedJavaStreams }
-import org.apache.flink.streaming.api.datastream.{ KeyedStream => KeyedJavaStream }
-import language.implicitConversions
-
-package object scala {
-  // We have this here so that we always have generated TypeInformationS when
-  // using the Scala API
-  implicit def createTypeInformation[T]: TypeInformation[T] = macro TypeUtils.createTypeInfo[T]
-
-  implicit def javaToScalaStream[R](javaStream: JavaStream[R]): DataStream[R] =
-    new DataStream[R](javaStream)
-    
-  implicit def javaToScalaGroupedStream[R, K](javaStream: KeyedJavaStream[R, K]):
-  KeyedStream[R, K] = new KeyedStream[R, K](javaStream)
-
-  implicit def javaToScalaSplitStream[R](javaStream: SplitJavaStream[R]): SplitStream[R] =
-    new SplitStream[R](javaStream)
-
-  implicit def javaToScalaConnectedStream[IN1, IN2](javaStream: ConnectedJavaStreams[IN1, IN2]):
-  ConnectedStreams[IN1, IN2] = new ConnectedStreams[IN1, IN2](javaStream)
-
-  implicit def seqToFlinkSource[T: ClassTag: TypeInformation](scalaSeq: Seq[T]) : DataStream[T] =
-    StreamExecutionEnvironment.getExecutionEnvironment.fromCollection(scalaSeq)
-
-  private[flink] def fieldNames2Indices(
-      typeInfo: TypeInformation[_],
-      fields: Array[String]): Array[Int] = {
-    typeInfo match {
-      case ti: CaseClassTypeInfo[_] =>
-        val result = ti.getFieldIndices(fields)
-
-        if (result.contains(-1)) {
-          throw new IllegalArgumentException("Fields '" + fields.mkString(", ") +
-            "' are not valid for '" + ti.toString + "'.")
-        }
-
-        result
-
-      case _ =>
-        throw new UnsupportedOperationException("Specifying fields by name is only" +
-          "supported on Case Classes (for now).")
-    }
-  }
-
-  def createTuple2TypeInformation[T1, T2](
-      t1: TypeInformation[T1],
-      t2: TypeInformation[T2]) : TypeInformation[(T1, T2)] =
-    apiTupleCreator[T1, T2](t1, t2)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java
deleted file mode 100644
index 0c60719..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.scala.api;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.scala.OutputFormatTestPrograms;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.util.Collector;
-
-public class CsvOutputFormatITCase extends StreamingProgramTestBase {
-
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		OutputFormatTestPrograms.wordCountToCsv(WordCountData.TEXT, resultPath);
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		//Strip the parentheses from the expected text like output
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES
-				.replaceAll("[\\\\(\\\\)]", ""), resultPath);
-	}
-
-	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(String value, Collector<Tuple2<String, Integer>> out)
-				throws Exception {
-			// normalize and split the line
-			String[] tokens = value.toLowerCase().split("\\W+");
-
-			// emit the pairs
-			for (String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Tuple2<String, Integer>(token, 1));
-				}
-			}
-		}
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/SocketOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/SocketOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/SocketOutputFormatITCase.java
deleted file mode 100644
index a2a78b7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/SocketOutputFormatITCase.java
+++ /dev/null
@@ -1,36 +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.scala.api;
-
-import org.apache.flink.streaming.api.scala.OutputFormatTestPrograms;
-import org.apache.flink.streaming.util.SocketOutputTestBase;
-import org.apache.flink.streaming.util.SocketProgramITCaseBase;
-import org.apache.flink.test.testdata.WordCountData;
-import org.junit.Ignore;
-
-@Ignore
-//This test sometimes failes most likely due to the behaviour
-//of the socket. Disabled for now.
-public class SocketOutputFormatITCase extends SocketOutputTestBase {
-
-		@Override
-		protected void testProgram() throws Exception {
-			OutputFormatTestPrograms.wordCountToSocket(WordCountData.TEXT, HOST, port);
-		}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java b/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java
deleted file mode 100644
index 731222e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/StatefulFunctionITCase.java
+++ /dev/null
@@ -1,30 +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.scala.api;
-
-import org.apache.flink.streaming.api.scala.StateTestPrograms;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-
-public class StatefulFunctionITCase extends StreamingProgramTestBase {
-
-	@Override
-	protected void testProgram() throws Exception {
-		StateTestPrograms.testStatefulFunctions();
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java b/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java
deleted file mode 100644
index 530ba67..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.scala.api;
-
-import org.apache.flink.streaming.api.scala.OutputFormatTestPrograms;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class TextOutputFormatITCase extends StreamingProgramTestBase {
-
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		resultPath = getTempDirPath("result");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		OutputFormatTestPrograms.wordCountToText(WordCountData.TEXT, resultPath);
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
deleted file mode 100644
index 7da7bc3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
+++ /dev/null
@@ -1,265 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.api.common.functions.RichReduceFunction
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.streaming.api.TimeCharacteristic
-import org.apache.flink.streaming.api.functions.windowing.{WindowFunction, AllWindowFunction}
-import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.streaming.api.windowing.assigners.{TumblingTimeWindows, SlidingTimeWindows}
-import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
-import org.apache.flink.streaming.api.windowing.time.Time
-import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, CountTrigger}
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow
-import org.apache.flink.streaming.runtime.operators.windowing.buffers.{HeapWindowBuffer, PreAggregatingHeapWindowBuffer}
-import org.apache.flink.streaming.runtime.operators.windowing._
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.apache.flink.util.Collector
-
-import org.junit.Assert._
-import org.junit.{Ignore, Test}
-
-class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
-
-  /**
-   * These tests ensure that the fast aligned time windows operator is used if the
-   * conditions are right.
-   *
-   * TODO: update once we have optimized aligned time windows operator for all-windows
-   */
-  @Ignore
-  @Test
-  def testFastTimeWindows(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime)
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .windowAll(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .reduce(reducer)
-
-    val transform1 = window1.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[AggregatingProcessingTimeWindowOperator[_, _]])
-
-    val window2 = source
-      .keyBy(0)
-      .windowAll(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
-      def apply(
-                    window: TimeWindow,
-                    values: java.lang.Iterable[(String, Int)],
-                    out: Collector[(String, Int)]) { }
-    })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[AccumulatingProcessingTimeWindowOperator[_, _, _]])
-  }
-
-  @Test
-  def testNonEvicting(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .windowAll(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
-      .reduce(reducer)
-
-    val transform1 = window1.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
-    val winOperator1 = operator1.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
-    assertTrue(winOperator1.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
-    assertTrue(
-      winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
-
-
-    val window2 = source
-      .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
-      .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
-      def apply(
-                    window: TimeWindow,
-                    values: java.lang.Iterable[(String, Int)],
-                    out: Collector[(String, Int)]) { }
-    })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
-    val winOperator2 = operator2.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
-    assertTrue(winOperator2.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
-    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
-  }
-
-  @Test
-  def testEvicting(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .windowAll(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
-      .reduce(reducer)
-
-    val transform1 = window1.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]])
-    val winOperator1 = operator1.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
-    assertTrue(winOperator1.getTrigger.isInstanceOf[ProcessingTimeTrigger])
-    assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
-    assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
-
-
-    val window2 = source
-      .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
-      .evictor(CountEvictor.of(1000))
-      .apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
-      def apply(
-                    window: TimeWindow,
-                    values: java.lang.Iterable[(String, Int)],
-                    out: Collector[(String, Int)]) { }
-    })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]])
-    val winOperator2 = operator2.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
-    assertTrue(winOperator2.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
-    assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
-  }
-
-  @Test
-  def testPreReduce(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-
-    val source = env.fromElements(("hello", 1), ("hello", 2))
-
-    val reducer = new DummyReducer
-
-    val window1 = source
-      .keyBy(0)
-      .window(SlidingTimeWindows.of(
-        Time.of(1, TimeUnit.SECONDS),
-        Time.of(100, TimeUnit.MILLISECONDS)))
-      .trigger(CountTrigger.of(100))
-      .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-        def apply(
-                   tuple: Tuple,
-                   window: TimeWindow,
-                   values: java.lang.Iterable[(String, Int)],
-                   out: Collector[(String, Int)]) { }
-      })
-
-    val transform1 = window1.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator1 = transform1.getOperator
-
-    assertTrue(operator1.isInstanceOf[WindowOperator[_, _, _, _]])
-    val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _]]
-    assertTrue(winOperator1.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
-    assertTrue(
-      winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
-
-
-    val window2 = source
-      .keyBy(0)
-      .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
-      .trigger(CountTrigger.of(100))
-      .apply(reducer, new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
-        def apply(
-                   tuple: Tuple,
-                   window: TimeWindow,
-                   values: java.lang.Iterable[(String, Int)],
-                   out: Collector[(String, Int)]) { }
-      })
-
-    val transform2 = window2.getJavaStream.getTransformation
-      .asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
-
-    val operator2 = transform2.getOperator
-
-    assertTrue(operator2.isInstanceOf[WindowOperator[_, _, _, _]])
-    val winOperator2 = operator2.asInstanceOf[WindowOperator[_, _, _, _]]
-    assertTrue(winOperator2.getTrigger.isInstanceOf[CountTrigger[_]])
-    assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
-    assertTrue(
-      winOperator2.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
-  }
-
-}
-
-// ------------------------------------------------------------------------
-//  UDFs
-// ------------------------------------------------------------------------
-
-class DummyReducer extends RichReduceFunction[(String, Int)] {
-  def reduce(value1: (String, Int), value2: (String, Int)): (String, Int) = {
-    value1
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
deleted file mode 100644
index 3c1e9c3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
+++ /dev/null
@@ -1,275 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.api.scala
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.streaming.api.TimeCharacteristic
-import org.apache.flink.streaming.api.functions.TimestampExtractor
-import org.apache.flink.streaming.api.functions.sink.SinkFunction
-import org.apache.flink.streaming.api.functions.source.SourceFunction
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows
-import org.apache.flink.streaming.api.windowing.time.Time
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Test
-import org.junit.Assert._
-
-import scala.collection.mutable
-
-class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testCoGroup(): Unit = {
-    CoGroupJoinITCase.testResults = mutable.MutableList()
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    env.setParallelism(1)
-
-    val source1 = env.addSource(new SourceFunction[(String, Int)]() {
-      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
-        ctx.collect(("a", 0))
-        ctx.collect(("a", 1))
-        ctx.collect(("a", 2))
-        ctx.collect(("b", 3))
-        ctx.collect(("b", 4))
-        ctx.collect(("b", 5))
-        ctx.collect(("a", 6))
-        ctx.collect(("a", 7))
-        ctx.collect(("a", 8))
-      }
-
-      def cancel() {
-      }
-    }).assignTimestamps(new CoGroupJoinITCase.Tuple2TimestampExtractor)
-
-    val source2 = env.addSource(new SourceFunction[(String, Int)]() {
-      def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
-        ctx.collect(("a", 0))
-        ctx.collect(("a", 1))
-        ctx.collect(("b", 3))
-        ctx.collect(("c", 6))
-        ctx.collect(("c", 7))
-        ctx.collect(("c", 8))
-      }
-
-      def cancel() {
-      }
-    }).assignTimestamps(new CoGroupJoinITCase.Tuple2TimestampExtractor)
-
-    source1.coGroup(source2)
-      .where(_._1)
-      .equalTo(_._1)
-      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-      .apply { (first: Iterator[(String, Int)], second: Iterator[(String, Int)]) =>
-          "F:" + first.mkString("") + " S:" + second.mkString("")
-      }
-      .addSink(new SinkFunction[String]() {
-        def invoke(value: String) {
-          CoGroupJoinITCase.testResults += value
-        }
-      })
-
-    env.execute("CoGroup Test")
-
-    val expectedResult = mutable.MutableList(
-      "F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
-      "F:(b,3)(b,4)(b,5) S:(b,3)",
-      "F:(a,6)(a,7)(a,8) S:",
-      "F: S:(c,6)(c,7)(c,8)")
-
-    assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
-  }
-
-  @Test
-  def testJoin(): Unit = {
-    CoGroupJoinITCase.testResults = mutable.MutableList()
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    env.setParallelism(1)
-
-    val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
-      def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
-        ctx.collect(("a", "x", 0))
-        ctx.collect(("a", "y", 1))
-        ctx.collect(("a", "z", 2))
-
-        ctx.collect(("b", "u", 3))
-        ctx.collect(("b", "w", 5))
-
-        ctx.collect(("a", "i", 6))
-        ctx.collect(("a", "j", 7))
-        ctx.collect(("a", "k", 8))
-      }
-
-      def cancel() {
-      }
-    }).assignTimestamps(new CoGroupJoinITCase.Tuple3TimestampExtractor)
-
-    val source2 = env.addSource(new SourceFunction[(String, String, Int)]() {
-      def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
-        ctx.collect(("a", "u", 0))
-        ctx.collect(("a", "w", 1))
-
-        ctx.collect(("b", "i", 3))
-        ctx.collect(("b", "k", 5))
-
-        ctx.collect(("a", "x", 6))
-        ctx.collect(("a", "z", 8))
-      }
-
-      def cancel() {
-      }
-    }).assignTimestamps(new CoGroupJoinITCase.Tuple3TimestampExtractor)
-
-    source1.join(source2)
-      .where(_._1)
-      .equalTo(_._1)
-      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-      .apply( (l, r) => l.toString + ":" + r.toString)
-      .addSink(new SinkFunction[String]() {
-        def invoke(value: String) {
-          CoGroupJoinITCase.testResults += value
-        }
-      })
-
-    env.execute("Join Test")
-
-    val expectedResult = mutable.MutableList(
-      "(a,x,0):(a,u,0)",
-      "(a,x,0):(a,w,1)",
-      "(a,y,1):(a,u,0)",
-      "(a,y,1):(a,w,1)",
-      "(a,z,2):(a,u,0)",
-      "(a,z,2):(a,w,1)",
-      "(b,u,3):(b,i,3)",
-      "(b,u,3):(b,k,5)",
-      "(b,w,5):(b,i,3)",
-      "(b,w,5):(b,k,5)",
-      "(a,i,6):(a,x,6)",
-      "(a,i,6):(a,z,8)",
-      "(a,j,7):(a,x,6)",
-      "(a,j,7):(a,z,8)",
-      "(a,k,8):(a,x,6)",
-      "(a,k,8):(a,z,8)")
-
-    assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
-  }
-
-  @Test
-  def testSelfJoin(): Unit = {
-    CoGroupJoinITCase.testResults = mutable.MutableList()
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    env.setParallelism(1)
-
-    val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
-      def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
-        ctx.collect(("a", "x", 0))
-        ctx.collect(("a", "y", 1))
-        ctx.collect(("a", "z", 2))
-
-        ctx.collect(("b", "u", 3))
-        ctx.collect(("b", "w", 5))
-
-        ctx.collect(("a", "i", 6))
-        ctx.collect(("a", "j", 7))
-        ctx.collect(("a", "k", 8))
-      }
-
-      def cancel() {
-      }
-    }).assignTimestamps(new CoGroupJoinITCase.Tuple3TimestampExtractor)
-
-    source1.join(source1)
-      .where(_._1)
-      .equalTo(_._1)
-      .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-      .apply( (l, r) => l.toString + ":" + r.toString)
-      .addSink(new SinkFunction[String]() {
-      def invoke(value: String) {
-        CoGroupJoinITCase.testResults += value
-      }
-    })
-
-    env.execute("Self-Join Test")
-
-    val expectedResult = mutable.MutableList(
-      "(a,x,0):(a,x,0)",
-      "(a,x,0):(a,y,1)",
-      "(a,x,0):(a,z,2)",
-      "(a,y,1):(a,x,0)",
-      "(a,y,1):(a,y,1)",
-      "(a,y,1):(a,z,2)",
-      "(a,z,2):(a,x,0)",
-      "(a,z,2):(a,y,1)",
-      "(a,z,2):(a,z,2)",
-      "(b,u,3):(b,u,3)",
-      "(b,u,3):(b,w,5)",
-      "(b,w,5):(b,u,3)",
-      "(b,w,5):(b,w,5)",
-      "(a,i,6):(a,i,6)",
-      "(a,i,6):(a,j,7)",
-      "(a,i,6):(a,k,8)",
-      "(a,j,7):(a,i,6)",
-      "(a,j,7):(a,j,7)",
-      "(a,j,7):(a,k,8)",
-      "(a,k,8):(a,i,6)",
-      "(a,k,8):(a,j,7)",
-      "(a,k,8):(a,k,8)")
-
-    assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
-  }
-
-}
-
-
-object CoGroupJoinITCase {
-  private var testResults: mutable.MutableList[String] = null
-
-  private class Tuple2TimestampExtractor extends TimestampExtractor[(String, Int)] {
-    def extractTimestamp(element: (String, Int), currentTimestamp: Long): Long = {
-      element._2
-    }
-
-    def extractWatermark(element: (String, Int), currentTimestamp: Long): Long = {
-      element._2 - 1
-    }
-
-    def getCurrentWatermark: Long = {
-      Long.MinValue
-    }
-  }
-
-  private class Tuple3TimestampExtractor extends TimestampExtractor[(String, String, Int)] {
-    def extractTimestamp(element: (String, String, Int), currentTimestamp: Long): Long = {
-      element._3
-    }
-
-    def extractWatermark(element: (String, String, Int), currentTimestamp: Long): Long = {
-      element._3 - 1
-    }
-
-    def getCurrentWatermark: Long = {
-      Long.MinValue
-    }
-  }
-}


[05/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
new file mode 100644
index 0000000..591ef51
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/PojoExample.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.wordcount;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Collector;
+
+/**
+ * This example shows an implementation of WordCount without using the Tuple2
+ * type, but a custom class.
+ * 
+ * <p>
+ * Usage: <code>WordCount &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}.
+ * 
+ * <p>
+ * This example shows how to:
+ * <ul>
+ * <li>use POJO data types,
+ * <li>write a simple Flink program,
+ * <li>write and use user-defined functions. 
+ * </ul>
+ */
+public class PojoExample {
+	
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		DataStream<String> text = getTextDataStream(env);
+
+		DataStream<Word> counts =
+		// split up the lines into Word objects
+		text.flatMap(new Tokenizer())
+		// group by the field word and sum up the frequency
+				.keyBy("word").sum("frequency");
+
+		if (fileOutput) {
+			counts.writeAsText(outputPath);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("WordCount Pojo Example");
+	}
+
+	// *************************************************************************
+	// DATA TYPES
+	// *************************************************************************
+
+	/**
+	 * This is the POJO (Plain Old Java Object) that is being used for all the
+	 * operations. As long as all fields are public or have a getter/setter, the
+	 * system can handle them
+	 */
+	public static class Word {
+
+		private String word;
+		private Integer frequency;
+
+		public Word() {
+		}
+
+		public Word(String word, int i) {
+			this.word = word;
+			this.frequency = i;
+		}
+
+		public String getWord() {
+			return word;
+		}
+
+		public void setWord(String word) {
+			this.word = word;
+		}
+
+		public Integer getFrequency() {
+			return frequency;
+		}
+
+		public void setFrequency(Integer frequency) {
+			this.frequency = frequency;
+		}
+
+		@Override
+		public String toString() {
+			return "(" + word + "," + frequency + ")";
+		}
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Implements the string tokenizer that splits sentences into words as a
+	 * user-defined FlatMapFunction. The function takes a line (String) and
+	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
+	 * Integer>).
+	 */
+	public static final class Tokenizer implements FlatMapFunction<String, Word> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(String value, Collector<Word> out) {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new Word(token, 1));
+				}
+			}
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: PojoExample <text path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing PojoExample example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from a file.");
+			System.out.println("  Usage: PojoExample <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		} else {
+			// get default test text data
+			return env.fromElements(WordCountData.WORDS);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
new file mode 100644
index 0000000..a594c94
--- /dev/null
+++ b/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.examples.wordcount;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.Collector;
+
+/**
+ * Implements the "WordCount" program that computes a simple word occurrence
+ * histogram over text files in a streaming fashion.
+ * 
+ * <p>
+ * The input is a plain text file with lines separated by newline characters.
+ * 
+ * <p>
+ * Usage: <code>WordCount &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}.
+ * 
+ * <p>
+ * This example shows how to:
+ * <ul>
+ * <li>write a simple Flink Streaming program,
+ * <li>use tuple data types,
+ * <li>write and use user-defined functions.
+ * </ul>
+ * 
+ */
+public class WordCount {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		DataStream<String> text = getTextDataStream(env);
+
+		DataStream<Tuple2<String, Integer>> counts =
+		// split up the lines in pairs (2-tuples) containing: (word,1)
+		text.flatMap(new Tokenizer())
+		// group by the tuple field "0" and sum up tuple field "1"
+				.keyBy(0).sum(1);
+
+		// emit result
+		if (fileOutput) {
+			counts.writeAsText(outputPath);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("Streaming WordCount");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Implements the string tokenizer that splits sentences into words as a
+	 * user-defined FlatMapFunction. The function takes a line (String) and
+	 * splits it into multiple pairs in the form of "(word,1)" (Tuple2<String,
+	 * Integer>).
+	 */
+	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(String value, Collector<Tuple2<String, Integer>> out)
+				throws Exception {
+			// normalize and split the line
+			String[] tokens = value.toLowerCase().split("\\W+");
+
+			// emit the pairs
+			for (String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new Tuple2<String, Integer>(token, 1));
+				}
+			}
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: WordCount <text path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing WordCount example with built-in default data.");
+			System.out.println("  Provide parameters to read input data from a file.");
+			System.out.println("  Usage: WordCount <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		} else {
+			// get default test text data
+			return env.fromElements(WordCountData.WORDS);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala b/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
new file mode 100644
index 0000000..42484e8
--- /dev/null
+++ b/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.scala.examples.join
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.scala._
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows
+import org.apache.flink.streaming.api.windowing.time.Time
+
+import scala.Stream._
+import scala.language.postfixOps
+import scala.util.Random
+
+object WindowJoin {
+
+  // *************************************************************************
+  // PROGRAM
+  // *************************************************************************
+
+  case class Grade(time: Long, name: String, grade: Int)
+  case class Salary(time: Long, name: String, salary: Int)
+  case class Person(name: String, grade: Int, salary: Int)
+
+  def main(args: Array[String]) {
+
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+
+    //Create streams for grades and salaries by mapping the inputs to the corresponding objects
+    val grades = setGradesInput(env)
+    val salaries = setSalariesInput(env)
+
+    //Join the two input streams by name on the last 2 seconds every second and create new
+    //Person objects containing both grade and salary
+    val joined = grades.join(salaries)
+        .where(_.name)
+        .equalTo(_.name)
+        .window(SlidingTimeWindows.of(Time.of(2, TimeUnit.SECONDS), Time.of(1, TimeUnit.SECONDS)))
+        .apply { (g, s) => Person(g.name, g.grade, s.salary) }
+
+    if (fileOutput) {
+      joined.writeAsText(outputPath)
+    } else {
+      joined.print()
+    }
+
+    env.execute("WindowJoin")
+  }
+
+  // *************************************************************************
+  // USER FUNCTIONS
+  // *************************************************************************
+
+  val names = Array("tom", "jerry", "alice", "bob", "john", "grace")
+  val gradeCount = 5
+  val salaryMax = 10000
+  val sleepInterval = 100
+  
+  def gradeStream: Stream[(Long, String, Int)] = {
+    def gradeMapper(names: Array[String])(x: Int): (Long, String, Int) =
+      {
+        if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
+        (System.currentTimeMillis(),names(Random.nextInt(names.length)),Random.nextInt(gradeCount))
+      }
+    range(1, 100).map(gradeMapper(names))
+  }
+
+  def salaryStream: Stream[(Long, String, Int)] = {
+    def salaryMapper(x: Int): (Long, String, Int) =
+      {
+        if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
+        (System.currentTimeMillis(), names(Random.nextInt(names.length)), Random.nextInt(salaryMax))
+      }
+    range(1, 100).map(salaryMapper)
+  }
+
+  def parseMap(line : String): (Long, String, Int) = {
+    val record = line.substring(1, line.length - 1).split(",")
+    (record(0).toLong, record(1), record(2).toInt)
+  }
+
+  // *************************************************************************
+  // UTIL METHODS
+  // *************************************************************************
+
+  private var fileInput: Boolean = false
+  private var fileOutput: Boolean = false
+
+  private var gradesPath: String = null
+  private var salariesPath: String = null
+  private var outputPath: String = null
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      if (args.length == 1) {
+        fileOutput = true
+        outputPath = args(0)
+      }
+      else if (args.length == 3) {
+        fileInput = true
+        fileOutput = true
+        gradesPath = args(0)
+        salariesPath = args(1)
+        outputPath = args(2)
+      } else {
+        System.err.println("Usage: WindowJoin <result path> or WindowJoin <input path 1> " +
+          "<input path 2> <result path>")
+        return false
+      }
+    } else {
+      System.out.println("Executing WindowJoin with generated data.")
+      System.out.println("  Provide parameter to write to file.")
+      System.out.println("  Usage: WindowJoin <result path>")
+    }
+    true
+  }
+
+  private def setGradesInput(env: StreamExecutionEnvironment) : DataStream[Grade] = {
+    if (fileInput) {
+      env.readTextFile(gradesPath).map(parseMap _ ).map(x => Grade(x._1, x._2, x._3))
+    } else {
+      env.fromCollection(gradeStream).map(x => Grade(x._1, x._2, x._3))
+    }
+  }
+
+  private def setSalariesInput(env: StreamExecutionEnvironment) : DataStream[Salary] = {
+    if (fileInput) {
+      env.readTextFile(salariesPath).map(parseMap _).map(x => Salary(x._1, x._2, x._3))
+    }
+    else {
+      env.fromCollection(salaryStream).map(x => Salary(x._1, x._2, x._3))
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala b/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
new file mode 100644
index 0000000..9ec17d4
--- /dev/null
+++ b/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketTextStreamWordCount.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.scala.examples.socket
+
+import org.apache.flink.streaming.api.scala._
+
+import scala.language.postfixOps
+
+/**
+ * This example shows an implementation of WordCount with data from a text socket. 
+ * To run the example make sure that the service providing the text data is already up and running.
+ *
+ * To start an example socket text stream on your local machine run netcat from a command line, 
+ * where the parameter specifies the port number:
+ *
+ * {{{
+ *   nc -lk 9999
+ * }}}
+ *
+ * Usage:
+ * {{{
+ *   SocketTextStreamWordCount <hostname> <port> <output path>
+ * }}}
+ *
+ * This example shows how to:
+ *
+ *   - use StreamExecutionEnvironment.socketTextStream
+ *   - write a simple Flink Streaming program in scala.
+ *   - write and use user-defined functions.
+ */
+object SocketTextStreamWordCount {
+
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+
+    //Create streams for names and ages by mapping the inputs to the corresponding objects
+    val text = env.socketTextStream(hostName, port)
+    val counts = text.flatMap { _.toLowerCase.split("\\W+") filter { _.nonEmpty } }
+      .map { (_, 1) }
+      .keyBy(0)
+      .sum(1)
+
+    if (fileOutput) {
+      counts.writeAsText(outputPath, 1)
+    } else {
+      counts print
+    }
+
+    env.execute("Scala SocketTextStreamWordCount Example")
+  }
+
+  private def parseParameters(args: Array[String]): Boolean = {
+      if (args.length == 3) {
+        fileOutput = true
+        hostName = args(0)
+        port = args(1).toInt
+        outputPath = args(2)
+      } else if (args.length == 2) {
+        hostName = args(0)
+        port = args(1).toInt
+      } else {
+        System.err.println("Usage: SocketTextStreamWordCount <hostname> <port> [<output path>]")
+        return false
+      }
+    true
+  }
+
+  private var fileOutput: Boolean = false
+  private var hostName: String = null
+  private var port: Int = 0
+  private var outputPath: String = null
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala b/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
new file mode 100644
index 0000000..f26f32c
--- /dev/null
+++ b/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/windowing/TopSpeedWindowing.scala
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.scala.examples.windowing
+
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.functions.windowing.delta.DeltaFunction
+import org.apache.flink.streaming.api.scala._
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
+import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.triggers.DeltaTrigger
+
+import scala.Stream._
+import scala.math._
+import scala.language.postfixOps
+import scala.util.Random
+
+/**
+ * An example of grouped stream windowing where different eviction and 
+ * trigger policies can be used. A source fetches events from cars 
+ * every 1 sec containing their id, their current speed (kmh),
+ * overall elapsed distance (m) and a timestamp. The streaming
+ * example triggers the top speed of each car every x meters elapsed 
+ * for the last y seconds.
+ */
+object TopSpeedWindowing {
+
+  // *************************************************************************
+  // PROGRAM
+  // *************************************************************************
+
+  case class CarEvent(carId: Int, speed: Int, distance: Double, time: Long)
+
+  val numOfCars = 2
+  val evictionSec = 10
+  val triggerMeters = 50d
+
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    env.setParallelism(1)
+
+    val cars = setCarsInput(env)
+
+    val topSeed = cars
+      .assignAscendingTimestamps( _.time )
+      .keyBy("carId")
+      .window(GlobalWindows.create)
+      .evictor(TimeEvictor.of(Time.of(evictionSec * 1000, TimeUnit.MILLISECONDS)))
+      .trigger(DeltaTrigger.of(triggerMeters, new DeltaFunction[CarEvent] {
+        def getDelta(oldSp: CarEvent, newSp: CarEvent): Double = newSp.distance - oldSp.distance
+      }))
+//      .window(Time.of(evictionSec * 1000, (car : CarEvent) => car.time))
+//      .every(Delta.of[CarEvent](triggerMeters,
+//          (oldSp,newSp) => newSp.distance-oldSp.distance, CarEvent(0,0,0,0)))
+      .maxBy("speed")
+
+    if (fileOutput) {
+      topSeed.writeAsText(outputPath)
+    } else {
+      topSeed.print
+    }
+
+    env.execute("TopSpeedWindowing")
+
+  }
+
+  // *************************************************************************
+  // USER FUNCTIONS
+  // *************************************************************************
+
+  def genCarStream(): Stream[CarEvent] = {
+
+    def nextSpeed(carEvent : CarEvent) : CarEvent =
+    {
+      val next =
+        if (Random.nextBoolean) min(100, carEvent.speed + 5) else max(0, carEvent.speed - 5)
+      CarEvent(carEvent.carId, next, carEvent.distance + next/3.6d,System.currentTimeMillis)
+    }
+    def carStream(speeds : Stream[CarEvent]) : Stream[CarEvent] =
+    {
+      Thread.sleep(1000)
+      speeds.append(carStream(speeds.map(nextSpeed)))
+    }
+    carStream(range(0, numOfCars).map(CarEvent(_,50,0,System.currentTimeMillis())))
+  }
+
+  def parseMap(line : String): (Int, Int, Double, Long) = {
+    val record = line.substring(1, line.length - 1).split(",")
+    (record(0).toInt, record(1).toInt, record(2).toDouble, record(3).toLong)
+  }
+
+  // *************************************************************************
+  // UTIL METHODS
+  // *************************************************************************
+
+  var fileInput = false
+  var fileOutput = false
+  var inputPath : String = null
+  var outputPath : String = null
+
+  def parseParameters(args: Array[String]): Boolean = {
+    if (args.length > 0) {
+      if (args.length == 2) {
+        fileInput = true
+        fileOutput = true
+        inputPath = args(0)
+        outputPath = args(1)
+        true
+      } else {
+        System.err.println("Usage: TopSpeedWindowing <input path> <output path>")
+        false
+      }
+    } else {
+      true
+    }
+  }
+
+  private def setCarsInput(env: StreamExecutionEnvironment) : DataStream[CarEvent] = {
+    if (fileInput) {
+      env.readTextFile(inputPath).map(parseMap(_)).map(x => CarEvent(x._1, x._2, x._3, x._4))
+    } else {
+      env.fromCollection(genCarStream())
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java
new file mode 100644
index 0000000..07d6766
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/iteration/IterateExampleITCase.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.iteration;
+
+import org.apache.flink.streaming.examples.iteration.IterateExample;
+import org.apache.flink.streaming.examples.iteration.util.IterateExampleData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class IterateExampleITCase extends StreamingProgramTestBase {
+
+
+	protected String inputPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		inputPath = createTempFile("fibonacciInput.txt", IterateExampleData.INPUT_PAIRS);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(IterateExampleData.RESULTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		IterateExample.main(new String[]{inputPath, resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
new file mode 100644
index 0000000..e657b67
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.join;
+
+import org.apache.flink.streaming.examples.join.WindowJoin;
+import org.apache.flink.streaming.examples.join.util.WindowJoinData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class WindowJoinITCase extends StreamingProgramTestBase {
+
+	protected String gradesPath;
+	protected String salariesPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
+		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		// since the two sides of the join might have different speed
+		// the exact output can not be checked just whether it is well-formed
+		// checks that the result lines look like e.g. (bob, 2, 2015)
+		checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java
new file mode 100644
index 0000000..83569dc
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/ml/IncrementalLearningSkeletonITCase.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.ml;
+
+import org.apache.flink.streaming.examples.ml.IncrementalLearningSkeleton;
+import org.apache.flink.streaming.examples.ml.util.IncrementalLearningSkeletonData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class IncrementalLearningSkeletonITCase extends StreamingProgramTestBase {
+
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(IncrementalLearningSkeletonData.RESULTS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		IncrementalLearningSkeleton.main(new String[]{resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java
new file mode 100644
index 0000000..838834b
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/socket/SocketTextStreamWordCountITCase.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.socket;
+
+import org.apache.flink.streaming.examples.socket.SocketTextStreamWordCount;
+import org.apache.flink.streaming.util.SocketProgramITCaseBase;
+
+public class SocketTextStreamWordCountITCase extends SocketProgramITCaseBase {
+
+	@Override
+	protected void testProgram() throws Exception {
+		SocketTextStreamWordCount.main(new String[]{HOST, port.toString(), resultPath});
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java
new file mode 100644
index 0000000..7850082
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/twitter/TwitterStreamITCase.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.twitter;
+
+import org.apache.flink.streaming.examples.twitter.TwitterStream;
+import org.apache.flink.streaming.examples.twitter.util.TwitterStreamData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class TwitterStreamITCase extends StreamingProgramTestBase {
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(TwitterStreamData.STREAMING_COUNTS_AS_TUPLES, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		TwitterStream.main(new String[]{resultPath});
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java
new file mode 100644
index 0000000..7f46be9
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/SessionWindowingITCase.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.windowing;
+
+import org.apache.flink.streaming.examples.windowing.SessionWindowing;
+import org.apache.flink.streaming.examples.windowing.util.SessionWindowingData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class SessionWindowingITCase extends StreamingProgramTestBase {
+
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(SessionWindowingData.EXPECTED, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		SessionWindowing.main(new String[]{resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
new file mode 100644
index 0000000..37812c9
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/TopSpeedWindowingExampleITCase.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.windowing;
+
+import org.apache.flink.streaming.examples.windowing.TopSpeedWindowing;
+import org.apache.flink.streaming.examples.windowing.util.TopSpeedWindowingExampleData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class TopSpeedWindowingExampleITCase extends StreamingProgramTestBase {
+	
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		setParallelism(1); //needed to ensure total ordering for windows
+		textPath = createTempFile("text.txt", TopSpeedWindowingExampleData.CAR_DATA);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(TopSpeedWindowingExampleData.TOP_SPEEDS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		TopSpeedWindowing.main(new String[]{textPath, resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java
new file mode 100644
index 0000000..e7cce60
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/windowing/WindowWordCountITCase.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.windowing;
+
+import org.apache.flink.streaming.examples.windowing.WindowWordCount;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.test.testdata.WordCountData;
+
+public class WindowWordCountITCase extends StreamingProgramTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+	protected String windowSize = "250";
+	protected String slideSize = "150";
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		// since the parallel tokenizers might have different speed
+		// the exact output can not be checked just whether it is well-formed
+		// checks that the result lines look like e.g. (faust, 2)
+		checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d)+\\)");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WindowWordCount.main(new String[]{textPath, resultPath, windowSize, slideSize});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java
new file mode 100644
index 0000000..6e3c213
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/PojoExampleITCase.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.wordcount;
+
+import org.apache.flink.streaming.examples.wordcount.PojoExample;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.test.testdata.WordCountData;
+
+public class PojoExampleITCase extends StreamingProgramTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		PojoExample.main(new String[]{textPath, resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java
new file mode 100644
index 0000000..fcf568e
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/wordcount/WordCountITCase.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleJavaPrograms.wordcount;
+
+import org.apache.flink.streaming.examples.wordcount.WordCount;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.test.testdata.WordCountData;
+
+public class WordCountITCase extends StreamingProgramTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		textPath = createTempFile("text.txt", WordCountData.TEXT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WordCount.main(new String[]{textPath, resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
new file mode 100644
index 0000000..08ce890
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleScalaPrograms.join;
+
+import org.apache.flink.streaming.scala.examples.join.WindowJoin;
+import org.apache.flink.streaming.examples.join.util.WindowJoinData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class WindowJoinITCase extends StreamingProgramTestBase {
+
+	protected String gradesPath;
+	protected String salariesPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
+		salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		// since the two sides of the join might have different speed
+		// the exact output can not be checked just whether it is well-formed
+		// checks that the result lines look like e.g. Person(bob, 2, 2015)
+		checkLinesAgainstRegexp(resultPath, "^Person\\([a-z]+,(\\d),(\\d)+\\)");
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java
new file mode 100644
index 0000000..b3629ad
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/socket/SocketTextStreamWordCountITCase.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleScalaPrograms.socket;
+
+import org.apache.flink.streaming.scala.examples.socket.SocketTextStreamWordCount;
+import org.apache.flink.streaming.util.SocketProgramITCaseBase;
+
+public class SocketTextStreamWordCountITCase extends SocketProgramITCaseBase {
+
+	@Override
+	protected void testProgram() throws Exception {
+		SocketTextStreamWordCount.main(new String[]{HOST, port.toString(), resultPath});
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java
new file mode 100644
index 0000000..ef4e47f
--- /dev/null
+++ b/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/windowing/TopSpeedWindowingExampleITCase.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.test.exampleScalaPrograms.windowing;
+
+import org.apache.flink.streaming.scala.examples.windowing.TopSpeedWindowing;
+import org.apache.flink.streaming.examples.windowing.util.TopSpeedWindowingExampleData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class TopSpeedWindowingExampleITCase extends StreamingProgramTestBase {
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		setParallelism(1); //needed to ensure total ordering for windows
+		textPath = createTempFile("text.txt", TopSpeedWindowingExampleData.CAR_DATA);
+		resultPath = getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(TopSpeedWindowingExampleData.TOP_CASE_CLASS_SPEEDS, resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		TopSpeedWindowing.main(new String[]{textPath, resultPath});
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml
new file mode 100644
index 0000000..25d4031
--- /dev/null
+++ b/flink-streaming-java/pom.xml
@@ -0,0 +1,116 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-streaming-java</artifactId>
+	<name>flink-streaming-java</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-math</artifactId>
+			<version>2.2</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.sling</groupId>
+			<artifactId>org.apache.sling.commons.json</artifactId>
+			<version>2.0.6</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-math3</artifactId>
+            <version>3.5</version>
+        </dependency>
+
+    </dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
+			<!-- disable fork reuse for the streaming project, because of
+			incorrect declaration of tests -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<reuseForks>false</reuseForks>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java
new file mode 100644
index 0000000..db46d00
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/CheckpointingMode.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api;
+
+/**
+ * The checkpointing mode defines what consistency guarantees the system gives in the presence of
+ * failures.
+ * 
+ * <p>When checkpointing is activated, the data streams are replayed such that lost parts of the
+ * processing are repeated. For stateful operations and functions, the checkpointing mode defines
+ * whether the system draws checkpoints such that a recovery behaves as if the operators/functions
+ * see each record "exactly once" ({@link #EXACTLY_ONCE}), or whether the checkpoints are drawn
+ * in a simpler fashion that typically encounteres some duplicates upon recovery
+ * ({@link #AT_LEAST_ONCE})</p> 
+ */
+public enum CheckpointingMode {
+
+	/**
+	 * Sets the checkpointing mode to "exactly once". This mode means that the system will
+	 * checkpoint the operator and user function state in such a way that, upon recovery,
+	 * every record will be reflected exactly once in the operator state.
+	 * 
+	 * <p>For example, if a user function counts the number of elements in a stream, 
+	 * this number will consistently be equal to the number of actual elements in the stream,
+	 * regardless of failures and recovery.</p>
+	 * 
+	 * <p>Note that this does not mean that each record flows through the streaming data flow
+	 * only once. It means that upon recovery, the state of operators/functions is restored such
+	 * that the resumed data streams pick up exactly at after the last modification to the state.</p> 
+	 *  
+	 * <p>Note that this mode does not guarantee exactly-once behavior in the interaction with
+	 * external systems (only state in Flink's operators and user functions). The reason for that
+	 * is that a certain level of "collaboration" is required between two systems to achieve
+	 * exactly-once guarantees. However, for certain systems, connectors can be written that facilitate
+	 * this collaboration.</p>
+	 * 
+	 * <p>This mode sustains high throughput. Depending on the data flow graph and operations,
+	 * this mode may increase the record latency, because operators need to align their input
+	 * streams, in order to create a consistent snapshot point. The latency increase for simple
+	 * dataflows (no repartitioning) is negligible. For simple dataflows with repartitioning, the average
+	 * latency remains small, but the slowest records typically have an increased latency.</p>
+	 */
+	EXACTLY_ONCE,
+
+	/**
+	 * Sets the checkpointing mode to "at least once". This mode means that the system will
+	 * checkpoint the operator and user function state in a simpler way. Upon failure and recovery,
+	 * some records may be reflected multiple times in the operator state.
+	 * 
+	 * <p>For example, if a user function counts the number of elements in a stream, 
+	 * this number will equal to, or larger, than the actual number of elements in the stream,
+	 * in the presence of failure and recovery.</p>
+	 * 
+	 * <p>This mode has minimal impact on latency and may be preferable in very-low latency
+	 * scenarios, where a sustained very-low latency (such as few milliseconds) is needed,
+	 * and where occasional duplicate messages (on recovery) do not matter.</p>
+	 */
+	AT_LEAST_ONCE
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java
new file mode 100644
index 0000000..125ca65
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/TimeCharacteristic.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api;
+
+/**
+ * The time characteristic defines how the system determines time for time-dependent
+ * order and operations that depend on time (such as time windows).
+ */
+public enum TimeCharacteristic {
+
+	/**
+	 * Processing time for operators means that the operator uses the system clock of the machine
+	 * to determine the current time of the data stream. Processing-time windows trigger based
+	 * on wall-clock time and include whatever elements happen to have arrived at the operator at
+	 * that point in time.
+	 * <p>
+	 * Using processing time for window operations results in general in quite non-deterministic results,
+	 * because the contents of the windows depends on the speed in which elements arrive. It is, however,
+	 * the cheapest method of forming windows and the method that introduces the least latency.
+	 */
+	ProcessingTime,
+
+	/**
+	 * Ingestion time means that the time of each individual element in the stream is determined
+	 * when the element enters the Flink streaming data flow. Operations like windows group the
+	 * elements based on that time, meaning that processing speed within the streaming dataflow
+	 * does not affect windowing, but only the speed at which sources receive elements.
+	 * <p>
+	 * Ingestion time is often a good compromise between processing time and event time.
+	 * It does not need and special manual form of watermark generation, and events are typically
+	 * not too much out-or-order when they arrive at operators; in fact, out-of-orderness can 
+	 * only be introduced by streaming shuffles or split/join/union operations. The fact that elements
+	 * are not very much out-of-order means that the latency increase is moderate, compared to event
+	 * time.
+	 */
+	IngestionTime,
+
+	/**
+	 * Event time means that the time of each individual element in the stream (also called event)
+	 * is determined by the event's individual custom timestamp. These timestamps either exist in the
+	 * elements from before they entered the Flink streaming dataflow, or are user-assigned at the sources.
+	 * The big implication of this is that elements arrive in the sources and in all operators generally
+	 * out of order, meaning that elements with earlier timestamps may arrive after elements with
+	 * later timestamps.
+	 * <p>
+	 * Operators that window or order data with respect to event time must buffer data until they can
+	 * be sure that all timestamps for a certain time interval have been received. This is handled by
+	 * the so called "time watermarks".
+	 * <p>
+	 * Operations based on event time are very predictable - the result of windowing operations
+	 * is typically identical no matter when the window is executed and how fast the streams operate.
+	 * At the same time, the buffering and tracking of event time is also costlier than operating
+	 * with processing time, and typically also introduces more latency. The amount of extra
+	 * cost depends mostly on how much out of order the elements arrive, i.e., how long the time span
+	 * between the arrival of early and late elements is. With respect to the "time watermarks", this
+	 * means that the cost typically depends on how early or late the watermarks can be generated
+	 * for their timestamp.
+	 * <p>
+	 * In relation to {@link #IngestionTime}, the event time is similar, but refers the the event's
+	 * original time, rather than the time assigned at the data source. Practically, that means that
+	 * event time has generally more meaning, but also that it takes longer to determine that all
+	 * elements for a certain time have arrived.
+	 */
+	EventTime
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java
new file mode 100644
index 0000000..c2d2182
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointNotifier.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.streaming.api.checkpoint;
+
+/**
+ * This interface must be implemented by functions/operations that want to receive
+ * a commit notification once a checkpoint has been completely acknowledged by all
+ * participants.
+ */
+public interface CheckpointNotifier {
+
+	/**
+	 * This method is called as a notification once a distributed checkpoint has been completed.
+	 * 
+	 * Note that any exception during this method will not cause the checkpoint to
+	 * fail any more.
+	 * 
+	 * @param checkpointId The ID of the checkpoint that has been completed.
+	 * @throws Exception
+	 */
+	void notifyCheckpointComplete(long checkpointId) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
new file mode 100644
index 0000000..ac1cbfb
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/Checkpointed.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.checkpoint;
+
+import java.io.Serializable;
+
+/**
+ * This method must be implemented by functions that have state that needs to be
+ * checkpointed. The functions get a call whenever a checkpoint should take place
+ * and return a snapshot of their state, which will be checkpointed.
+ * 
+ * <p>This interface marks a function as <i>synchronously</i> checkpointed. While the
+ * state is written, the function is not called, so the function needs not return a
+ * copy of its state, but may return a reference to its state. Functions that can
+ * continue to work and mutate the state, even while the state snapshot is being accessed,
+ * can implement the {@link org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously}
+ * interface.</p>
+ * 
+ * @param <T> The type of the operator state.
+ */
+public interface Checkpointed<T extends Serializable> {
+
+	/**
+	 * Gets the current state of the function of operator. The state must reflect the result of all
+	 * prior invocations to this function. 
+	 * 
+	 * @param checkpointId The ID of the checkpoint.
+	 * @param checkpointTimestamp The timestamp of the checkpoint, as derived by
+	 *                            System.currentTimeMillis() on the JobManager.
+	 *                            
+	 * @return A snapshot of the operator state.
+	 * 
+	 * @throws Exception Thrown if the creation of the state object failed. This causes the
+	 *                   checkpoint to fail. The system may decide to fail the operation (and trigger
+	 *                   recovery), or to discard this checkpoint attempt and to continue running
+	 *                   and to try again with the next checkpoint attempt.
+	 */
+	T snapshotState(long checkpointId, long checkpointTimestamp) throws Exception;
+
+	/**
+	 * Restores the state of the function or operator to that of a previous checkpoint.
+	 * This method is invoked when a function is executed as part of a recovery run.
+	 *
+	 * Note that restoreState() is called before open().
+	 *
+	 * @param state The state to be restored. 
+	 */
+	void restoreState(T state) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
new file mode 100644
index 0000000..4bd89c4
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/CheckpointedAsynchronously.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.checkpoint;
+
+import java.io.Serializable;
+
+/**
+ * This interface marks a function/operator as <i>asynchronously checkpointed</i>.
+ * Similar to the {@link Checkpointed} interface, the function must produce a
+ * snapshot of its state. However, the function must be able to continue working
+ * and mutating its state without mutating the returned state snapshot.
+ * 
+ * <p>Asynchronous checkpoints are desirable, because they allow the data streams at the
+ * point of the checkpointed function/operator to continue running while the checkpoint
+ * is in progress.</p>
+ * 
+ * <p>To be able to support asynchronous snapshots, the state returned by the
+ * {@link #snapshotState(long, long)} method is typically a copy or shadow copy
+ * of the actual state.</p>
+ */
+public interface CheckpointedAsynchronously<T extends Serializable> extends Checkpointed<T> {}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
new file mode 100644
index 0000000..7034b11
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.collector.selector;
+
+import java.util.ArrayList;
+
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Collector;
+
+public class BroadcastOutputSelectorWrapper<OUT> implements OutputSelectorWrapper<OUT> {
+
+	private static final long serialVersionUID = 1L;
+	
+	private final ArrayList<Collector<StreamRecord<OUT>>> outputs;
+
+	public BroadcastOutputSelectorWrapper() {
+		outputs = new ArrayList<Collector<StreamRecord<OUT>>>();
+	}
+	
+	@Override
+	public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge) {
+		outputs.add(output);
+	}
+
+	@Override
+	public Iterable<Collector<StreamRecord<OUT>>> getSelectedOutputs(OUT record) {
+		return outputs;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
new file mode 100644
index 0000000..84558fc
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.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.api.collector.selector;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.util.Collector;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DirectedOutputSelectorWrapper<OUT> implements OutputSelectorWrapper<OUT> {
+
+	private static final long serialVersionUID = 1L;
+
+	private static final Logger LOG = LoggerFactory.getLogger(DirectedOutputSelectorWrapper.class);
+
+	private List<OutputSelector<OUT>> outputSelectors;
+
+	private HashMap<String, ArrayList<Collector<StreamRecord<OUT>>>> outputMap;
+	private HashSet<Collector<StreamRecord<OUT>>> selectAllOutputs;
+
+	public DirectedOutputSelectorWrapper(List<OutputSelector<OUT>> outputSelectors) {
+		this.outputSelectors = outputSelectors;
+		this.selectAllOutputs = new HashSet<Collector<StreamRecord<OUT>>>();
+		this.outputMap = new HashMap<String, ArrayList<Collector<StreamRecord<OUT>>>>();
+	}
+	
+	@Override
+	public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge) {
+		List<String> selectedNames = edge.getSelectedNames();
+
+		if (selectedNames.isEmpty()) {
+			selectAllOutputs.add(output);
+		}
+		else {
+			for (String selectedName : selectedNames) {
+				if (!outputMap.containsKey(selectedName)) {
+					outputMap.put(selectedName, new ArrayList<Collector<StreamRecord<OUT>>>());
+					outputMap.get(selectedName).add(output);
+				}
+				else {
+					if (!outputMap.get(selectedName).contains(output)) {
+						outputMap.get(selectedName).add(output);
+					}
+				}
+			}
+		}
+	}
+
+	@Override
+	public Iterable<Collector<StreamRecord<OUT>>> getSelectedOutputs(OUT record) {
+		Set<Collector<StreamRecord<OUT>>> selectedOutputs = new HashSet<Collector<StreamRecord<OUT>>>(selectAllOutputs);
+
+		for (OutputSelector<OUT> outputSelector : outputSelectors) {
+			Iterable<String> outputNames = outputSelector.select(record);
+
+			for (String outputName : outputNames) {
+				List<Collector<StreamRecord<OUT>>> outputList = outputMap.get(outputName);
+
+				try {
+					selectedOutputs.addAll(outputList);
+				} catch (NullPointerException e) {
+					if (LOG.isErrorEnabled()) {
+						String format = String.format(
+								"Cannot emit because no output is selected with the name: %s",
+								outputName);
+						LOG.error(format);
+					}
+				}
+			}
+		}
+
+		return selectedOutputs;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
new file mode 100644
index 0000000..9c6eede
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelector.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.api.collector.selector;
+
+import java.io.Serializable;
+
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.datastream.SplitStream;
+
+/**
+ * Interface for defining an OutputSelector for a {@link SplitStream} using
+ * the {@link SingleOutputStreamOperator#split} call. Every output object of a
+ * {@link SplitStream} will run through this operator to select outputs.
+ * 
+ * @param <OUT>
+ *            Type parameter of the split values.
+ */
+public interface OutputSelector<OUT> extends Serializable {
+	/**
+	 * Method for selecting output names for the emitted objects when using the
+	 * {@link SingleOutputStreamOperator#split} method. The values will be
+	 * emitted only to output names which are contained in the returned
+	 * iterable.
+	 * 
+	 * @param value
+	 *            Output object for which the output selection should be made.
+	 */
+	public Iterable<String> select(OUT value);
+}


[48/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
deleted file mode 100644
index 8066b3c..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
+++ /dev/null
@@ -1,689 +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.connectors.kafka;
-
-import kafka.cluster.Broker;
-import kafka.common.ErrorMapping;
-import kafka.javaapi.PartitionMetadata;
-import kafka.javaapi.TopicMetadata;
-import kafka.javaapi.TopicMetadataRequest;
-import kafka.javaapi.consumer.SimpleConsumer;
-import org.apache.commons.collections.map.LinkedMap;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.connectors.kafka.internals.Fetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.OffsetHandler;
-import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import org.apache.flink.util.NetUtils;
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.Node;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
- * Apache Kafka. The consumer can run in multiple parallel instances, each of which will pull
- * data from one or more Kafka partitions. 
- * 
- * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
- * during a failure, and that the computation processes elements "exactly once". 
- * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
- * 
- * <p>To support a variety of Kafka brokers, protocol versions, and offset committing approaches,
- * the Flink Kafka Consumer can be parametrized with a <i>fetcher</i> and an <i>offset handler</i>.</p>
- *
- * <h1>Fetcher</h1>
- * 
- * <p>The fetcher is responsible to pull data from Kafka. Because Kafka has undergone a change in
- * protocols and APIs, there are currently two fetchers available:</p>
- * 
- * <ul>
- *     <li>{@link FetcherType#NEW_HIGH_LEVEL}: A fetcher based on the new Kafka consumer API.
- *         This fetcher is generally more robust, but works only with later versions of
- *         Kafka (> 0.8.2).</li>
- *         
- *     <li>{@link FetcherType#LEGACY_LOW_LEVEL}: A fetcher based on the old low-level consumer API.
- *         This fetcher is works also with older versions of Kafka (0.8.1). The fetcher interprets
- *         the old Kafka consumer properties, like:
- *         <ul>
- *             <li>socket.timeout.ms</li>
- *             <li>socket.receive.buffer.bytes</li>
- *             <li>fetch.message.max.bytes</li>
- *             <li>auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)</li>
- *             <li>fetch.wait.max.ms</li>
- *         </ul>
- *     </li>
- * </ul>
- * 
- * <h1>Offset handler</h1>
- * 
- * <p>Offsets whose records have been read and are checkpointed will be committed back to Kafka / ZooKeeper
- * by the offset handler. In addition, the offset handler finds the point where the source initially
- * starts reading from the stream, when the streaming job is started.</p>
- * 
- * <p>Currently, the source offers two different offset handlers exist:</p>
- * <ul>
- *     <li>{@link OffsetStore#KAFKA}: Use this offset handler when the Kafka brokers are managing the offsets,
- *         and hence offsets need to be committed the Kafka brokers, rather than to ZooKeeper.
- *         Note that this offset handler works only on new versions of Kafka (0.8.2.x +) and
- *         with the {@link FetcherType#NEW_HIGH_LEVEL} fetcher.</li>
- *         
- *     <li>{@link OffsetStore#FLINK_ZOOKEEPER}: Use this offset handler when the offsets are managed
- *         by ZooKeeper, as in older versions of Kafka (0.8.1.x)</li>
- * </ul>
- * 
- * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
- * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
- * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
- * has consumed a topic.</p>
- * 
- * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
- * is constructed. That means that the client that submits the program needs to be able to
- * reach the Kafka brokers or ZooKeeper.</p>
- */
-public class FlinkKafkaConsumer<T> extends RichParallelSourceFunction<T>
-		implements CheckpointNotifier, CheckpointedAsynchronously<long[]>, ResultTypeQueryable<T> {
-
-	/**
-	 * The offset store defines how acknowledged offsets are committed back to Kafka. Different
-	 * options include letting Flink periodically commit to ZooKeeper, or letting Kafka manage the
-	 * offsets (new Kafka versions only).
-	 */
-	public enum OffsetStore {
-
-		/**
-		 * Let Flink manage the offsets. Flink will periodically commit them to Zookeeper (usually after
-		 * successful checkpoints), in the same structure as Kafka 0.8.2.x
-		 * 
-		 * <p>Use this mode when using the source with Kafka 0.8.1.x brokers.</p>
-		 */
-		FLINK_ZOOKEEPER,
-
-		/**
-		 * Use the mechanisms in Kafka to commit offsets. Depending on the Kafka configuration, different
-		 * mechanism will be used (broker coordinator, zookeeper)
-		 */ 
-		KAFKA
-	}
-
-	/**
-	 * The fetcher type defines which code paths to use to pull data from teh Kafka broker.
-	 */
-	public enum FetcherType {
-
-		/**
-		 * The legacy fetcher uses Kafka's old low-level consumer API.
-		 * 
-		 * <p>Use this fetcher for Kafka 0.8.1 brokers.</p>
-		 */
-		LEGACY_LOW_LEVEL,
-
-		/**
-		 * This fetcher uses a backport of the new consumer API to pull data from the Kafka broker.
-		 * It is the fetcher that will be maintained in the future, and it already 
-		 * handles certain failure cases with less overhead than the legacy fetcher.
-		 * 
-		 * <p>This fetcher works only Kafka 0.8.2 and 0.8.3 (and future versions).</p>
-		 */
-		NEW_HIGH_LEVEL
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private static final long serialVersionUID = -6272159445203409112L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer.class);
-
-	/** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid),
-	 * and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */
-	public static final long OFFSET_NOT_SET = -915623761776L;
-
-	/** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */
-	public static final int MAX_NUM_PENDING_CHECKPOINTS = 100;
-
-	/** Configuration key for the number of retries for getting the partition info */
-	public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry";
-
-	/** Default number of retries for getting the partition info. One retry means going through the full list of brokers */
-	public static final int DEFAULT_GET_PARTITIONS_RETRIES = 3;
-
-	
-	
-	// ------  Configuration of the Consumer -------
-	
-	/** The offset store where this consumer commits safe offsets */
-	private final OffsetStore offsetStore;
-
-	/** The type of fetcher to be used to pull data from Kafka */
-	private final FetcherType fetcherType;
-	
-	/** name of the topic consumed by this source */
-	private final String topic;
-	
-	/** The properties to parametrize the Kafka consumer and ZooKeeper client */ 
-	private final Properties props;
-	
-	/** The ids of the partitions that are read by this consumer */
-	private final int[] partitions;
-	
-	/** The schema to convert between Kafka#s byte messages, and Flink's objects */
-	private final DeserializationSchema<T> valueDeserializer;
-
-	// ------  Runtime State  -------
-
-	/** Data for pending but uncommitted checkpoints */
-	private final LinkedMap pendingCheckpoints = new LinkedMap();
-	
-	/** The fetcher used to pull data from the Kafka brokers */
-	private transient Fetcher fetcher;
-	
-	/** The committer that persists the committed offsets */
-	private transient OffsetHandler offsetHandler;
-	
-	/** The partitions actually handled by this consumer */
-	private transient List<TopicPartition> subscribedPartitions;
-
-	/** The offsets of the last returned elements */
-	private transient long[] lastOffsets;
-
-	/** The latest offsets that have been committed to Kafka or ZooKeeper. These are never
-	 * newer then the last offsets (Flink's internal view is fresher) */
-	private transient long[] commitedOffsets;
-	
-	/** The offsets to restore to, if the consumer restores state from a checkpoint */
-	private transient long[] restoreToOffset;
-	
-	private volatile boolean running = true;
-	
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler.
-	 * 
-	 * <p>To determine which kink of fetcher and offset handler to use, please refer to the docs
-	 * at the beginnign of this class.</p>
-	 * 
-	 * @param topic 
-	 *           The Kafka topic to read from.
-	 * @param valueDeserializer
-	 *           The deserializer to turn raw byte messages into Java/Scala objects.
-	 * @param props
-	 *           The properties that are used to configure both the fetcher and the offset handler.
-	 * @param offsetStore
-	 *           The type of offset store to use (Kafka / ZooKeeper)
-	 * @param fetcherType
-	 *           The type of fetcher to use (new high-level API, old low-level API).
-	 */
-	public FlinkKafkaConsumer(String topic, DeserializationSchema<T> valueDeserializer, Properties props, 
-								OffsetStore offsetStore, FetcherType fetcherType) {
-		this.offsetStore = checkNotNull(offsetStore);
-		this.fetcherType = checkNotNull(fetcherType);
-
-		if(fetcherType == FetcherType.NEW_HIGH_LEVEL) {
-			throw new UnsupportedOperationException("The fetcher for Kafka 0.8.3 is not yet " +
-					"supported in Flink");
-		}
-		if (offsetStore == OffsetStore.KAFKA && fetcherType == FetcherType.LEGACY_LOW_LEVEL) {
-			throw new IllegalArgumentException(
-					"The Kafka offset handler cannot be used together with the old low-level fetcher.");
-		}
-		
-		this.topic = checkNotNull(topic, "topic");
-		this.props = checkNotNull(props, "props");
-		this.valueDeserializer = checkNotNull(valueDeserializer, "valueDeserializer");
-
-		// validate the zookeeper properties
-		if (offsetStore == OffsetStore.FLINK_ZOOKEEPER) {
-			validateZooKeeperConfig(props);
-		}
-		
-		// Connect to a broker to get the partitions
-		List<PartitionInfo> partitionInfos = getPartitionsForTopic(topic, props);
-
-		// get initial partitions list. The order of the partitions is important for consistent 
-		// partition id assignment in restart cases.
-		this.partitions = new int[partitionInfos.size()];
-		for (int i = 0; i < partitionInfos.size(); i++) {
-			partitions[i] = partitionInfos.get(i).partition();
-			
-			if (partitions[i] >= partitions.length) {
-				throw new RuntimeException("Kafka partition numbers are sparse");
-			}
-		}
-		LOG.info("Topic {} has {} partitions", topic, partitions.length);
-
-		// make sure that we take care of the committing
-		props.setProperty("enable.auto.commit", "false");
-	}
-
-	// ------------------------------------------------------------------------
-	//  Source life cycle
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		
-		final int numConsumers = getRuntimeContext().getNumberOfParallelSubtasks();
-		final int thisComsumerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		
-		// pick which partitions we work on
-		subscribedPartitions = assignPartitions(this.partitions, this.topic, numConsumers, thisComsumerIndex);
-		
-		if (LOG.isInfoEnabled()) {
-			LOG.info("Kafka consumer {} will read partitions {} out of partitions {}",
-					thisComsumerIndex, subscribedPartitions, Arrays.toString(partitions));
-		}
-
-		// we leave the fetcher as null, if we have no partitions
-		if (subscribedPartitions.isEmpty()) {
-			LOG.info("Kafka consumer {} has no partitions (empty source)", thisComsumerIndex);
-			return;
-		}
-		
-		// create fetcher
-		switch (fetcherType){
-			case NEW_HIGH_LEVEL:
-				throw new UnsupportedOperationException("Currently unsupported");
-			case LEGACY_LOW_LEVEL:
-				fetcher = new LegacyFetcher(topic, props, getRuntimeContext().getTaskName());
-				break;
-			default:
-				throw new RuntimeException("Requested unknown fetcher " + fetcher);
-		}
-		fetcher.setPartitionsToRead(subscribedPartitions);
-
-		// offset handling
-		switch (offsetStore){
-			case FLINK_ZOOKEEPER:
-				offsetHandler = new ZookeeperOffsetHandler(props);
-				break;
-			case KAFKA:
-				throw new Exception("Kafka offset handler cannot work with legacy fetcher");
-			default:
-				throw new RuntimeException("Requested unknown offset store " + offsetStore);
-		}
-		
-		// set up operator state
-		lastOffsets = new long[partitions.length];
-		commitedOffsets = new long[partitions.length];
-		
-		Arrays.fill(lastOffsets, OFFSET_NOT_SET);
-		Arrays.fill(commitedOffsets, OFFSET_NOT_SET);
-		
-		// seek to last known pos, from restore request
-		if (restoreToOffset != null) {
-			if (LOG.isInfoEnabled()) {
-				LOG.info("Consumer {} found offsets from previous checkpoint: {}",
-						thisComsumerIndex,  Arrays.toString(restoreToOffset));
-			}
-			
-			for (int i = 0; i < restoreToOffset.length; i++) {
-				long restoredOffset = restoreToOffset[i];
-				if (restoredOffset != OFFSET_NOT_SET) {
-					// if this fails because we are not subscribed to the topic, then the
-					// partition assignment is not deterministic!
-					
-					// we set the offset +1 here, because seek() is accepting the next offset to read,
-					// but the restore offset is the last read offset
-					fetcher.seek(new TopicPartition(topic, i), restoredOffset + 1);
-					lastOffsets[i] = restoredOffset;
-				}
-			}
-		}
-		else {
-			// no restore request. Let the offset handler take care of the initial offset seeking
-			offsetHandler.seekFetcherToInitialOffsets(subscribedPartitions, fetcher);
-		}
-	}
-
-	@Override
-	public void run(SourceContext<T> sourceContext) throws Exception {
-		if (fetcher != null) {
-			fetcher.run(sourceContext, valueDeserializer, lastOffsets);
-		}
-		else {
-			// this source never completes
-			final Object waitLock = new Object();
-			while (running) {
-				// wait until we are canceled
-				try {
-					//noinspection SynchronizationOnLocalVariableOrMethodParameter
-					synchronized (waitLock) {
-						waitLock.wait();
-					}
-				}
-				catch (InterruptedException e) {
-					// do nothing, check our "running" status
-				}
-			}
-		}
-		
-		// close the context after the work was done. this can actually only
-		// happen when the fetcher decides to stop fetching
-		sourceContext.close();
-	}
-
-	@Override
-	public void cancel() {
-		// set ourselves as not running
-		running = false;
-		
-		// close the fetcher to interrupt any work
-		Fetcher fetcher = this.fetcher;
-		this.fetcher = null;
-		if (fetcher != null) {
-			try {
-				fetcher.close();
-			}
-			catch (IOException e) {
-				LOG.warn("Error while closing Kafka connector data fetcher", e);
-			}
-		}
-		
-		OffsetHandler offsetHandler = this.offsetHandler;
-		this.offsetHandler = null;
-		if (offsetHandler != null) {
-			try {
-				offsetHandler.close();
-			}
-			catch (IOException e) {
-				LOG.warn("Error while closing Kafka connector offset handler", e);
-			}
-		}
-	}
-
-	@Override
-	public void close() throws Exception {
-		cancel();
-		super.close();
-	}
-
-	@Override
-	public TypeInformation<T> getProducedType() {
-		return valueDeserializer.getProducedType();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpoint and restore
-	// ------------------------------------------------------------------------
-
-	@Override
-	public long[] snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		if (lastOffsets == null) {
-			LOG.debug("snapshotState() requested on not yet opened source; returning null.");
-			return null;
-		}
-		if (!running) {
-			LOG.debug("snapshotState() called on closed source");
-			return null;
-		}
-
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}",
-					Arrays.toString(lastOffsets), checkpointId, checkpointTimestamp);
-		}
-
-		long[] currentOffsets = Arrays.copyOf(lastOffsets, lastOffsets.length);
-
-		// the map cannot be asynchronously updated, because only one checkpoint call can happen
-		// on this function at a time: either snapshotState() or notifyCheckpointComplete()
-		pendingCheckpoints.put(checkpointId, currentOffsets);
-			
-		while (pendingCheckpoints.size() > MAX_NUM_PENDING_CHECKPOINTS) {
-			pendingCheckpoints.remove(0);
-		}
-
-		return currentOffsets;
-	}
-
-	@Override
-	public void restoreState(long[] restoredOffsets) {
-		restoreToOffset = restoredOffsets;
-	}
-
-	@Override
-	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		if (fetcher == null) {
-			LOG.debug("notifyCheckpointComplete() called on uninitialized source");
-			return;
-		}
-		if (!running) {
-			LOG.debug("notifyCheckpointComplete() called on closed source");
-			return;
-		}
-		
-		// only one commit operation must be in progress
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Committing offsets externally for checkpoint {}", checkpointId);
-		}
-
-		try {
-			long[] checkpointOffsets;
-	
-			// the map may be asynchronously updates when snapshotting state, so we synchronize
-			synchronized (pendingCheckpoints) {
-				final int posInMap = pendingCheckpoints.indexOf(checkpointId);
-				if (posInMap == -1) {
-					LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
-					return;
-				}
-	
-				checkpointOffsets = (long[]) pendingCheckpoints.remove(posInMap);
-				
-				// remove older checkpoints in map
-				for (int i = 0; i < posInMap; i++) {
-					pendingCheckpoints.remove(0);
-				}
-			}
-	
-			if (LOG.isInfoEnabled()) {
-				LOG.info("Committing offsets {} to offset store: {}", Arrays.toString(checkpointOffsets), offsetStore);
-			}
-	
-			// build the map of (topic,partition) -> committed offset
-			Map<TopicPartition, Long> offsetsToCommit = new HashMap<>();
-			for (TopicPartition tp : subscribedPartitions) {
-				
-				int partition = tp.partition();
-				long offset = checkpointOffsets[partition];
-				long lastCommitted = commitedOffsets[partition];
-				
-				if (offset != OFFSET_NOT_SET) {
-					if (offset > lastCommitted) {
-						offsetsToCommit.put(tp, offset);
-						LOG.debug("Committing offset {} for partition {}", offset, partition);
-					}
-					else {
-						LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, partition);
-					}
-				}
-			}
-			
-			offsetHandler.commit(offsetsToCommit);
-		}
-		catch (Exception e) {
-			if (running) {
-				throw e;
-			}
-			// else ignore exception if we are no longer running
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Miscellaneous utilities 
-	// ------------------------------------------------------------------------
-
-	protected static List<TopicPartition> assignPartitions(int[] partitions, String topicName,
-															int numConsumers, int consumerIndex) {
-		checkArgument(numConsumers > 0);
-		checkArgument(consumerIndex < numConsumers);
-		
-		List<TopicPartition> partitionsToSub = new ArrayList<>();
-
-		for (int i = 0; i < partitions.length; i++) {
-			if (i % numConsumers == consumerIndex) {
-				partitionsToSub.add(new TopicPartition(topicName, partitions[i]));
-			}
-		}
-		return partitionsToSub;
-	}
-	
-	// ------------------------------------------------------------------------
-	//  Kafka / ZooKeeper communication utilities
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Send request to Kafka to get partitions for topic.
-	 * 
-	 * @param topic The name of the topic.
-	 * @param properties The properties for the Kafka Consumer that is used to query the partitions for the topic. 
-	 */
-	public static List<PartitionInfo> getPartitionsForTopic(final String topic, final Properties properties) {
-		String seedBrokersConfString = properties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
-		final int numRetries = Integer.valueOf(properties.getProperty(GET_PARTITIONS_RETRIES_KEY, Integer.toString(DEFAULT_GET_PARTITIONS_RETRIES)));
-
-		checkNotNull(seedBrokersConfString, "Configuration property " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + " not set");
-		String[] seedBrokers = seedBrokersConfString.split(",");
-		List<PartitionInfo> partitions = new ArrayList<>();
-
-		Random rnd = new Random();
-		retryLoop: for(int retry = 0; retry < numRetries; retry++) {
-			// we pick a seed broker randomly to avoid overloading the first broker with all the requests when the
-			// parallel source instances start. Still, we try all available brokers.
-			int index = rnd.nextInt(seedBrokers.length);
-			brokersLoop: for (int arrIdx = 0; arrIdx < seedBrokers.length; arrIdx++) {
-				String seedBroker = seedBrokers[index];
-				LOG.info("Trying to get topic metadata from broker {} in try {}/{}", seedBroker, retry, numRetries);
-				if (++index == seedBrokers.length) {
-					index = 0;
-				}
-
-				URL brokerUrl = NetUtils.getCorrectHostnamePort(seedBroker);
-				SimpleConsumer consumer = null;
-				try {
-					final String clientId = "flink-kafka-consumer-partition-lookup";
-					final int soTimeout = Integer.valueOf(properties.getProperty("socket.timeout.ms", "30000"));
-					final int bufferSize = Integer.valueOf(properties.getProperty("socket.receive.buffer.bytes", "65536"));
-					consumer = new SimpleConsumer(brokerUrl.getHost(), brokerUrl.getPort(), soTimeout, bufferSize, clientId);
-
-					List<String> topics = Collections.singletonList(topic);
-					TopicMetadataRequest req = new TopicMetadataRequest(topics);
-					kafka.javaapi.TopicMetadataResponse resp = consumer.send(req);
-
-					List<TopicMetadata> metaData = resp.topicsMetadata();
-
-					// clear in case we have an incomplete list from previous tries
-					partitions.clear();
-					for (TopicMetadata item : metaData) {
-						if (item.errorCode() != ErrorMapping.NoError()) {
-							if (item.errorCode() == ErrorMapping.InvalidTopicCode() || item.errorCode() == ErrorMapping.UnknownTopicOrPartitionCode()) {
-								// fail hard if topic is unknown
-								throw new RuntimeException("Requested partitions for unknown topic", ErrorMapping.exceptionFor(item.errorCode()));
-							}
-							// warn and try more brokers
-							LOG.warn("Error while getting metadata from broker " + seedBroker + " to find partitions for " + topic,
-									ErrorMapping.exceptionFor(item.errorCode()));
-							continue brokersLoop;
-						}
-						if (!item.topic().equals(topic)) {
-							LOG.warn("Received metadata from topic " + item.topic() + " even though it was not requested. Skipping ...");
-							continue brokersLoop;
-						}
-						for (PartitionMetadata part : item.partitionsMetadata()) {
-							Node leader = brokerToNode(part.leader());
-							Node[] replicas = new Node[part.replicas().size()];
-							for (int i = 0; i < part.replicas().size(); i++) {
-								replicas[i] = brokerToNode(part.replicas().get(i));
-							}
-
-							Node[] ISRs = new Node[part.isr().size()];
-							for (int i = 0; i < part.isr().size(); i++) {
-								ISRs[i] = brokerToNode(part.isr().get(i));
-							}
-							PartitionInfo pInfo = new PartitionInfo(topic, part.partitionId(), leader, replicas, ISRs);
-							partitions.add(pInfo);
-						}
-					}
-					break retryLoop; // leave the loop through the brokers
-				} catch (Exception e) {
-					LOG.warn("Error communicating with broker " + seedBroker + " to find partitions for " + topic, e);
-				} finally {
-					if (consumer != null) {
-						consumer.close();
-					}
-				}
-			} // brokers loop
-		} // retries loop
-		return partitions;
-	}
-
-	private static Node brokerToNode(Broker broker) {
-		return new Node(broker.id(), broker.host(), broker.port());
-	}
-	
-	protected static void validateZooKeeperConfig(Properties props) {
-		if (props.getProperty("zookeeper.connect") == null) {
-			throw new IllegalArgumentException("Required property 'zookeeper.connect' has not been set in the properties");
-		}
-		if (props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-			throw new IllegalArgumentException("Required property '" + ConsumerConfig.GROUP_ID_CONFIG
-					+ "' has not been set in the properties");
-		}
-		
-		try {
-			//noinspection ResultOfMethodCallIgnored
-			Integer.parseInt(props.getProperty("zookeeper.session.timeout.ms", "0"));
-		}
-		catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Property 'zookeeper.session.timeout.ms' is not a valid integer");
-		}
-		
-		try {
-			//noinspection ResultOfMethodCallIgnored
-			Integer.parseInt(props.getProperty("zookeeper.connection.timeout.ms", "0"));
-		}
-		catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Property 'zookeeper.connection.timeout.ms' is not a valid integer");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
deleted file mode 100644
index 21f24e6..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.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.connectors.kafka;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Creates a Kafka consumer compatible with reading from Kafka 0.8.1.x brokers.
- * The consumer will internally use the old low-level Kafka API, and manually commit offsets
- * partition offsets to ZooKeeper.
- * 
- * <p>The following additional configuration values are available:</p>
- * <ul>
- *   <li>socket.timeout.ms</li>
- *   <li>socket.receive.buffer.bytes</li>
- *   <li>fetch.message.max.bytes</li>
- *   <li>auto.offset.reset with the values "latest", "earliest" (unlike 0.8.2 behavior)</li>
- *   <li>fetch.wait.max.ms</li>
- * </ul>
- * 
- * @param <T> The type of elements produced by this consumer.
- */
-public class FlinkKafkaConsumer081<T> extends FlinkKafkaConsumer<T> {
-
-	private static final long serialVersionUID = -5649906773771949146L;
-
-	/**
-	 * Creates a new Kafka 0.8.1.x streaming source consumer.
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer081(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
deleted file mode 100644
index 77e41e5..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-import java.util.Properties;
-
-/**
- * Creates a Kafka consumer compatible with reading from Kafka 0.8.2.x brokers.
- * The consumer will internally use the old low-level Kafka API, and manually commit offsets
- * partition offsets to ZooKeeper.
- *
- * Once Kafka released the new consumer with Kafka 0.8.3 Flink might use the 0.8.3 consumer API
- * also against Kafka 0.8.2 installations.
- *
- * @param <T> The type of elements produced by this consumer.
- */
-public class FlinkKafkaConsumer082<T> extends FlinkKafkaConsumer<T> {
-
-	private static final long serialVersionUID = -8450689820627198228L;
-
-	/**
-	 * Creates a new Kafka 0.8.2.x streaming source consumer.
-	 * 
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects. 
-	 * @param props
-	 *           The properties used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public FlinkKafkaConsumer082(String topic, DeserializationSchema<T> valueDeserializer, Properties props) {
-		super(topic, valueDeserializer, props, OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
deleted file mode 100644
index 715f5ee..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
+++ /dev/null
@@ -1,288 +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.connectors.kafka;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.streaming.connectors.kafka.partitioner.FixedPartitioner;
-import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.util.NetUtils;
-
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.serialization.ByteArraySerializer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.Properties;
-
-
-/**
- * Flink Sink to produce data into a Kafka topic.
- *
- * Please note that this producer does not have any reliability guarantees.
- *
- * @param <IN> Type of the messages to write into Kafka.
- */
-public class FlinkKafkaProducer<IN> extends RichSinkFunction<IN>  {
-
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class);
-
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Array with the partition ids of the given topicId
-	 * The size of this array is the number of partitions
-	 */
-	private final int[] partitions;
-
-	/**
-	 * User defined properties for the Producer
-	 */
-	private final Properties producerConfig;
-
-	/**
-	 * The name of the topic this producer is writing data to
-	 */
-	private final String topicId;
-
-	/**
-	 * (Serializable) SerializationSchema for turning objects used with Flink into
-	 * byte[] for Kafka.
-	 */
-	private final SerializationSchema<IN, byte[]> schema;
-
-	/**
-	 * User-provided partitioner for assigning an object to a Kafka partition.
-	 */
-	private final KafkaPartitioner partitioner;
-
-	/**
-	 * Flag indicating whether to accept failures (and log them), or to fail on failures
-	 */
-	private boolean logFailuresOnly;
-	
-	// -------------------------------- Runtime fields ------------------------------------------
-
-	/** KafkaProducer instance */
-	private transient KafkaProducer<byte[], byte[]> producer;
-
-	/** The callback than handles error propagation or logging callbacks */
-	private transient Callback callback;
-	
-	/** Errors encountered in the async producer are stored here */
-	private transient volatile Exception asyncException;
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param brokerList
-	 *			Comma separated addresses of the brokers
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema.
-	 */
-	public FlinkKafkaProducer(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema) {
-		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), null);
-	}
-
-	/**
-	 * Creates a FlinkKafkaProducer for a given topic. The sink produces its input to
-	 * the topic.
-	 *
-	 * @param topicId
-	 * 			ID of the Kafka topic.
-	 * @param serializationSchema
-	 * 			User defined serialization schema.
-	 * @param producerConfig
-	 * 			Properties with the producer configuration.
-	 */
-	public FlinkKafkaProducer(String topicId, SerializationSchema<IN, byte[]> serializationSchema, Properties producerConfig) {
-		this(topicId, serializationSchema, producerConfig, null);
-	}
-
-	/**
-	 * The main constructor for creating a FlinkKafkaProducer.
-	 *
-	 * @param topicId The topic to write data to
-	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[]
-	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
-	 * @param customPartitioner A serializable partitioner for assining messages to Kafka partitions.
-	 */
-	public FlinkKafkaProducer(String topicId, SerializationSchema<IN, byte[]> serializationSchema, Properties producerConfig, KafkaPartitioner customPartitioner) {
-		Preconditions.checkNotNull(topicId, "TopicID not set");
-		Preconditions.checkNotNull(serializationSchema, "serializationSchema not set");
-		Preconditions.checkNotNull(producerConfig, "producerConfig not set");
-		ClosureCleaner.ensureSerializable(customPartitioner);
-		ClosureCleaner.ensureSerializable(serializationSchema);
-
-		this.topicId = topicId;
-		this.schema = serializationSchema;
-		this.producerConfig = producerConfig;
-
-		// set the producer configuration properties.
-
-		if(!producerConfig.contains(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
-			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
-		} else {
-			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
-		}
-
-		if(!producerConfig.contains(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
-			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
-		} else {
-			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
-		}
-
-
-		// create a local KafkaProducer to get the list of partitions.
-		// this will also ensure locally that all required ProducerConfig values are set.
-		try (KafkaProducer<Void, IN> getPartitionsProd = new KafkaProducer<>(this.producerConfig)) {
-			List<PartitionInfo> partitionsList = getPartitionsProd.partitionsFor(topicId);
-
-			this.partitions = new int[partitionsList.size()];
-			for (int i = 0; i < partitions.length; i++) {
-				partitions[i] = partitionsList.get(i).partition();
-			}
-			getPartitionsProd.close();
-		}
-
-		if (customPartitioner == null) {
-			this.partitioner = new FixedPartitioner();
-		} else {
-			this.partitioner = customPartitioner;
-		}
-	}
-
-	// ---------------------------------- Properties --------------------------
-
-	/**
-	 * Defines whether the producer should fail on errors, or only log them.
-	 * If this is set to true, then exceptions will be only logged, if set to false,
-	 * exceptions will be eventually thrown and cause the streaming program to 
-	 * fail (and enter recovery).
-	 * 
-	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
-	 */
-	public void setLogFailuresOnly(boolean logFailuresOnly) {
-		this.logFailuresOnly = logFailuresOnly;
-	}
-
-	// ----------------------------------- Utilities --------------------------
-	
-	/**
-	 * Initializes the connection to Kafka.
-	 */
-	@Override
-	public void open(Configuration configuration) {
-		producer = new org.apache.kafka.clients.producer.KafkaProducer<>(this.producerConfig);
-
-		RuntimeContext ctx = getRuntimeContext();
-		partitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), partitions);
-
-		LOG.info("Starting FlinkKafkaProducer ({}/{}) to produce into topic {}", 
-				ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks(), topicId);
-		
-		if (logFailuresOnly) {
-			callback = new Callback() {
-				
-				@Override
-				public void onCompletion(RecordMetadata metadata, Exception e) {
-					if (e != null) {
-						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
-					}
-				}
-			};
-		}
-		else {
-			callback = new Callback() {
-				@Override
-				public void onCompletion(RecordMetadata metadata, Exception exception) {
-					if (exception != null && asyncException == null) {
-						asyncException = exception;
-					}
-				}
-			};
-		}
-	}
-
-	/**
-	 * Called when new data arrives to the sink, and forwards it to Kafka.
-	 *
-	 * @param next
-	 * 		The incoming data
-	 */
-	@Override
-	public void invoke(IN next) throws Exception {
-		// propagate asynchronous errors
-		checkErroneous();
-		
-		byte[] serialized = schema.serialize(next);
-		ProducerRecord<byte[], byte[]> record = new ProducerRecord<byte[], byte[]>(topicId,
-				partitioner.partition(next, partitions.length),
-				null, serialized);
-		
-		producer.send(record, callback);
-	}
-
-
-	@Override
-	public void close() throws Exception {
-		if (producer != null) {
-			producer.close();
-		}
-		
-		// make sure we propagate pending errors
-		checkErroneous();
-	}
-
-
-	// ----------------------------------- Utilities --------------------------
-
-	private void checkErroneous() throws Exception {
-		Exception e = asyncException;
-		if (e != null) {
-			// prevent double throwing
-			asyncException = null;
-			throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e);
-		}
-	}
-	
-	public static Properties getPropertiesFromBrokerList(String brokerList) {
-		String[] elements = brokerList.split(",");
-		for(String broker: elements) {
-			NetUtils.getCorrectHostnamePort(broker);
-		}
-		Properties props = new Properties();
-		props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList);
-		return props;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
deleted file mode 100644
index f856926..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/KafkaSink.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.streaming.connectors.kafka.api;
-
-
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-
-/**
- * Sink that emits its inputs to a Kafka topic.
- *
- * The KafkaSink has been relocated to org.apache.flink.streaming.connectors.kafka.KafkaSink.
- * This class will be removed in future releases of Flink.
- */
-@Deprecated
-public class KafkaSink<IN> extends FlinkKafkaProducer<IN> {
-	public KafkaSink(String brokerList, String topicId, SerializationSchema<IN, byte[]> serializationSchema) {
-		super(brokerList, topicId, serializationSchema);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
deleted file mode 100644
index 869c44f..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/api/persistent/PersistentKafkaSource.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.api.persistent;
-
-import kafka.consumer.ConsumerConfig;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
-
-/**
- * Creates a Kafka consumer compatible with reading from Kafka 0.8.1+ consumers.
- *
- * This class is provided as a migration path from the old Flink kafka connectors to the new, updated implemntations.
- *
- * Please use FlinkKafkaConsumer081 and FlinkKafkaConsumer082.
- *
- * @param <T> The type of elements produced by this consumer.
- */
-@Deprecated
-public class PersistentKafkaSource<T> extends FlinkKafkaConsumer<T> {
-
-	private static final long serialVersionUID = -8450689820627198228L;
-
-	/**
-	 * Creates a new Kafka 0.8.2.x streaming source consumer.
-	 *
-	 * @param topic
-	 *           The name of the topic that should be consumed.
-	 * @param valueDeserializer
-	 *           The de-/serializer used to convert between Kafka's byte messages and Flink's objects.
-	 * @param consumerConfig
-	 *           The consumer config used to configure the Kafka consumer client, and the ZooKeeper client.
-	 */
-	public PersistentKafkaSource(String topic, DeserializationSchema<T> valueDeserializer, ConsumerConfig consumerConfig) {
-		super(topic, valueDeserializer, consumerConfig.props().props(), OffsetStore.FLINK_ZOOKEEPER, FetcherType.LEGACY_LOW_LEVEL);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
deleted file mode 100644
index 4345926..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.kafka.common.TopicPartition;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * A fetcher pulls data from Kafka, from a fix set of partitions.
- * The fetcher supports "seeking" inside the partitions, i.e., moving to a different offset.
- */
-public interface Fetcher {
-
-	/**
-	 * Set which partitions the fetcher should pull from.
-	 * 
-	 * @param partitions The list of partitions for a topic that the fetcher will pull from.
-	 */
-	void setPartitionsToRead(List<TopicPartition> partitions);
-
-	/**
-	 * Closes the fetcher. This will stop any operation in the
-	 * {@link #run(SourceFunction.SourceContext, DeserializationSchema, long[])} method and eventually
-	 * close underlying connections and release all resources.
-	 */
-	void close() throws IOException;
-
-	/**
-	 * Starts fetch data from Kafka and emitting it into the stream.
-	 * 
-	 * <p>To provide exactly once guarantees, the fetcher needs emit a record and update the update
-	 * of the last consumed offset in one atomic operation:</p>
-	 * <pre>{@code
-	 * 
-	 * while (running) {
-	 *     T next = ...
-	 *     long offset = ...
-	 *     int partition = ...
-	 *     synchronized (sourceContext.getCheckpointLock()) {
-	 *         sourceContext.collect(next);
-	 *         lastOffsets[partition] = offset;
-	 *     }
-	 * }
-	 * }</pre>
-	 * 
-	 * @param sourceContext The source context to emit elements to.
-	 * @param valueDeserializer The deserializer to decode the raw values with.
-	 * @param lastOffsets The array into which to store the offsets foe which elements are emitted. 
-	 * 
-	 * @param <T> The type of elements produced by the fetcher and emitted to the source context.
-	 */
-	<T> void run(SourceFunction.SourceContext<T> sourceContext, DeserializationSchema<T> valueDeserializer, 
-					long[] lastOffsets) throws Exception;
-	
-	/**
-	 * Set the next offset to read from for the given partition.
-	 * For example, if the partition <i>i</i> offset is set to <i>n</i>, the Fetcher's next result
-	 * will be the message with <i>offset=n</i>.
-	 * 
-	 * @param topicPartition The partition for which to seek the offset.
-	 * @param offsetToRead To offset to seek to.
-	 */
-	void seek(TopicPartition topicPartition, long offsetToRead);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
deleted file mode 100644
index c4ba103..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
+++ /dev/null
@@ -1,622 +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.connectors.kafka.internals;
-
-import kafka.api.FetchRequestBuilder;
-import kafka.api.OffsetRequest;
-import kafka.api.PartitionOffsetRequestInfo;
-import kafka.common.ErrorMapping;
-import kafka.common.TopicAndPartition;
-import kafka.javaapi.FetchResponse;
-import kafka.javaapi.OffsetResponse;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.MessageAndOffset;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.util.StringUtils;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.Node;
-import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * This fetcher uses Kafka's low-level API to pull data from a specific
- * set of partitions and offsets for a certain topic.
- * 
- * <p>This code is in parts based on the tutorial code for the low-level Kafka consumer.</p>
- */
-public class LegacyFetcher implements Fetcher {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer.class);
-
-	/** The topic from which this fetcher pulls data */
-	private final String topic;
-	
-	/** The properties that configure the Kafka connection */
-	private final Properties config;
-	
-	/** The task name, to give more readable names to the spawned threads */
-	private final String taskName;
-	
-	/** The first error that occurred in a connection thread */
-	private final AtomicReference<Throwable> error;
-
-	/** The partitions that the fetcher should read, with their starting offsets */
-	private Map<TopicPartition, Long> partitionsToRead;
-	
-	/** Reference the the thread that executed the run() method. */
-	private volatile Thread mainThread;
-	
-	/** Flag to shot the fetcher down */
-	private volatile boolean running = true;
-
-	public LegacyFetcher(String topic, Properties props, String taskName) {
-		this.config = checkNotNull(props, "The config properties cannot be null");
-		this.topic = checkNotNull(topic, "The topic cannot be null");
-		this.taskName = taskName;
-		this.error = new AtomicReference<>();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Fetcher methods
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public void setPartitionsToRead(List<TopicPartition> partitions) {
-		partitionsToRead = new HashMap<>(partitions.size());
-		for (TopicPartition tp: partitions) {
-			partitionsToRead.put(tp, FlinkKafkaConsumer.OFFSET_NOT_SET);
-		}
-	}
-
-	@Override
-	public void seek(TopicPartition topicPartition, long offsetToRead) {
-		if (partitionsToRead == null) {
-			throw new IllegalArgumentException("No partitions to read set");
-		}
-		if (!partitionsToRead.containsKey(topicPartition)) {
-			throw new IllegalArgumentException("Can not set offset on a partition (" + topicPartition
-					+ ") we are not going to read. Partitions to read " + partitionsToRead);
-		}
-		partitionsToRead.put(topicPartition, offsetToRead);
-	}
-	
-	@Override
-	public void close() {
-		// flag needs to be check by the run() method that creates the spawned threads
-		this.running = false;
-		
-		// all other cleanup is made by the run method itself
-	}
-
-	@Override
-	public <T> void run(SourceFunction.SourceContext<T> sourceContext, 
-						DeserializationSchema<T> valueDeserializer,
-						long[] lastOffsets) throws Exception {
-		
-		if (partitionsToRead == null || partitionsToRead.size() == 0) {
-			throw new IllegalArgumentException("No partitions set");
-		}
-		
-		// NOTE: This method is needs to always release all resources it acquires
-		
-		this.mainThread = Thread.currentThread();
-
-		LOG.info("Reading from partitions " + partitionsToRead + " using the legacy fetcher");
-		
-		// get lead broker for each partition
-		
-		// NOTE: The kafka client apparently locks itself in an infinite loop sometimes
-		// when it is interrupted, so we run it only in a separate thread.
-		// since it sometimes refuses to shut down, we resort to the admittedly harsh
-		// means of killing the thread after a timeout.
-		PartitionInfoFetcher infoFetcher = new PartitionInfoFetcher(topic, config);
-		infoFetcher.start();
-		
-		KillerWatchDog watchDog = new KillerWatchDog(infoFetcher, 60000);
-		watchDog.start();
-		
-		final List<PartitionInfo> allPartitionsInTopic = infoFetcher.getPartitions();
-		
-		// brokers to fetch partitions from.
-		int fetchPartitionsCount = 0;
-		Map<Node, List<FetchPartition>> fetchBrokers = new HashMap<>();
-		
-		for (PartitionInfo partitionInfo : allPartitionsInTopic) {
-			if (partitionInfo.leader() == null) {
-				throw new RuntimeException("Unable to consume partition " + partitionInfo.partition()
-						+ " from topic "+partitionInfo.topic()+" because it does not have a leader");
-			}
-			
-			for (Map.Entry<TopicPartition, Long> entry : partitionsToRead.entrySet()) {
-				final TopicPartition topicPartition = entry.getKey();
-				final long offset = entry.getValue();
-				
-				// check if that partition is for us
-				if (topicPartition.partition() == partitionInfo.partition()) {
-					List<FetchPartition> partitions = fetchBrokers.get(partitionInfo.leader());
-					if (partitions == null) {
-						partitions = new ArrayList<>();
-						fetchBrokers.put(partitionInfo.leader(), partitions);
-					}
-					
-					partitions.add(new FetchPartition(topicPartition.partition(), offset));
-					fetchPartitionsCount++;
-					
-				}
-				// else this partition is not for us
-			}
-		}
-		
-		if (partitionsToRead.size() != fetchPartitionsCount) {
-			throw new RuntimeException(partitionsToRead.size() + " partitions to read, but got only "
-					+ fetchPartitionsCount + " partition infos with lead brokers.");
-		}
-
-		// create SimpleConsumers for each broker
-		ArrayList<SimpleConsumerThread<?>> consumers = new ArrayList<>(fetchBrokers.size());
-		
-		for (Map.Entry<Node, List<FetchPartition>> brokerInfo : fetchBrokers.entrySet()) {
-			final Node broker = brokerInfo.getKey();
-			final List<FetchPartition> partitionsList = brokerInfo.getValue();
-			
-			FetchPartition[] partitions = partitionsList.toArray(new FetchPartition[partitionsList.size()]);
-
-			SimpleConsumerThread<T> thread = new SimpleConsumerThread<>(this, config, topic,
-					broker, partitions, sourceContext, valueDeserializer, lastOffsets);
-
-			thread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)",
-					taskName, broker.id(), broker.host(), broker.port()));
-			thread.setDaemon(true);
-			consumers.add(thread);
-		}
-		
-		// last check whether we should abort.
-		if (!running) {
-			return;
-		}
-		
-		// start all consumer threads
-		for (SimpleConsumerThread<?> t : consumers) {
-			LOG.info("Starting thread {}", t.getName());
-			t.start();
-		}
-		
-		// wait until all consumer threads are done, or until we are aborted, or until
-		// an error occurred in one of the fetcher threads
-		try {
-			boolean someConsumersRunning = true;
-			while (running && error.get() == null && someConsumersRunning) {
-				try {
-					// wait for the consumer threads. if an error occurs, we are interrupted
-					for (SimpleConsumerThread<?> t : consumers) {
-						t.join();
-					}
-	
-					// safety net
-					someConsumersRunning = false;
-					for (SimpleConsumerThread<?> t : consumers) {
-						someConsumersRunning |= t.isAlive();
-					}
-				}
-				catch (InterruptedException e) {
-					// ignore. we should notice what happened in the next loop check
-				}
-			}
-			
-			// make sure any asynchronous error is noticed
-			Throwable error = this.error.get();
-			if (error != null) {
-				throw new Exception(error.getMessage(), error);
-			}
-		}
-		finally {
-			// make sure that in any case (completion, abort, error), all spawned threads are stopped
-			for (SimpleConsumerThread<?> t : consumers) {
-				if (t.isAlive()) {
-					t.cancel();
-				}
-			}
-		}
-	}
-	
-	/**
-	 * Reports an error from a fetch thread. This will cause the main thread to see this error,
-	 * abort, and cancel all other fetch threads.
-	 * 
-	 * @param error The error to report.
-	 */
-	void onErrorInFetchThread(Throwable error) {
-		if (this.error.compareAndSet(null, error)) {
-			// we are the first to report an error
-			if (mainThread != null) {
-				mainThread.interrupt();
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Representation of a partition to fetch.
-	 */
-	private static class FetchPartition {
-		
-		/** ID of the partition within the topic (0 indexed, as given by Kafka) */
-		int partition;
-		
-		/** Offset pointing at the next element to read from that partition. */
-		long nextOffsetToRead;
-
-		FetchPartition(int partition, long nextOffsetToRead) {
-			this.partition = partition;
-			this.nextOffsetToRead = nextOffsetToRead;
-		}
-		
-		@Override
-		public String toString() {
-			return "FetchPartition {partition=" + partition + ", offset=" + nextOffsetToRead + '}';
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Per broker fetcher
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Each broker needs its separate connection. This thread implements the connection to
-	 * one broker. The connection can fetch multiple partitions from the broker.
-	 * 
-	 * @param <T> The data type fetched.
-	 */
-	private static class SimpleConsumerThread<T> extends Thread {
-		
-		private final SourceFunction.SourceContext<T> sourceContext;
-		private final DeserializationSchema<T> valueDeserializer;
-		private final long[] offsetsState;
-		
-		private final FetchPartition[] partitions;
-		
-		private final Node broker;
-		private final String topic;
-		private final Properties config;
-
-		private final LegacyFetcher owner;
-
-		private SimpleConsumer consumer;
-		
-		private volatile boolean running = true;
-
-
-		// exceptions are thrown locally
-		public SimpleConsumerThread(LegacyFetcher owner,
-									Properties config, String topic,
-									Node broker,
-									FetchPartition[] partitions,
-									SourceFunction.SourceContext<T> sourceContext,
-									DeserializationSchema<T> valueDeserializer,
-									long[] offsetsState) {
-			this.owner = owner;
-			this.config = config;
-			this.topic = topic;
-			this.broker = broker;
-			this.partitions = partitions;
-			this.sourceContext = checkNotNull(sourceContext);
-			this.valueDeserializer = checkNotNull(valueDeserializer);
-			this.offsetsState = checkNotNull(offsetsState);
-		}
-
-		@Override
-		public void run() {
-			try {
-				// set up the config values
-				final String clientId = "flink-kafka-consumer-legacy-" + broker.id();
-
-				// these are the actual configuration values of Kafka + their original default values.
-				final int soTimeout = Integer.valueOf(config.getProperty("socket.timeout.ms", "30000"));
-				final int bufferSize = Integer.valueOf(config.getProperty("socket.receive.buffer.bytes", "65536"));
-				final int fetchSize = Integer.valueOf(config.getProperty("fetch.message.max.bytes", "1048576"));
-				final int maxWait = Integer.valueOf(config.getProperty("fetch.wait.max.ms", "100"));
-				final int minBytes = Integer.valueOf(config.getProperty("fetch.min.bytes", "1"));
-				
-				// create the Kafka consumer that we actually use for fetching
-				consumer = new SimpleConsumer(broker.host(), broker.port(), soTimeout, bufferSize, clientId);
-
-				// make sure that all partitions have some offsets to start with
-				// those partitions that do not have an offset from a checkpoint need to get
-				// their start offset from ZooKeeper
-				{
-					List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<>();
-
-					for (FetchPartition fp : partitions) {
-						if (fp.nextOffsetToRead == FlinkKafkaConsumer.OFFSET_NOT_SET) {
-							// retrieve the offset from the consumer
-							partitionsToGetOffsetsFor.add(fp);
-						}
-					}
-					if (partitionsToGetOffsetsFor.size() > 0) {
-						getLastOffset(consumer, topic, partitionsToGetOffsetsFor, getInvalidOffsetBehavior(config));
-						LOG.info("No prior offsets found for some partitions in topic {}. Fetched the following start offsets {}",
-								topic, partitionsToGetOffsetsFor);
-					}
-				}
-				
-				// Now, the actual work starts :-)
-				int OffsetOutOfRangeCount = 0;
-				while (running) {
-					FetchRequestBuilder frb = new FetchRequestBuilder();
-					frb.clientId(clientId);
-					frb.maxWait(maxWait);
-					frb.minBytes(minBytes);
-					
-					for (FetchPartition fp : partitions) {
-						frb.addFetch(topic, fp.partition, fp.nextOffsetToRead, fetchSize);
-					}
-					kafka.api.FetchRequest fetchRequest = frb.build();
-					LOG.debug("Issuing fetch request {}", fetchRequest);
-
-					FetchResponse fetchResponse;
-					fetchResponse = consumer.fetch(fetchRequest);
-
-					if (fetchResponse.hasError()) {
-						String exception = "";
-						List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<>();
-						for (FetchPartition fp : partitions) {
-							short code = fetchResponse.errorCode(topic, fp.partition);
-
-							if(code == ErrorMapping.OffsetOutOfRangeCode()) {
-								// we were asked to read from an out-of-range-offset (maybe set wrong in Zookeeper)
-								// Kafka's high level consumer is resetting the offset according to 'auto.offset.reset'
-								partitionsToGetOffsetsFor.add(fp);
-							} else if(code != ErrorMapping.NoError()) {
-								exception += "\nException for partition " + fp.partition + ": " +
-										StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
-							}
-						}
-						if (partitionsToGetOffsetsFor.size() > 0) {
-							// safeguard against an infinite loop.
-							if(OffsetOutOfRangeCount++ > 0) {
-								throw new RuntimeException("Found invalid offsets more than once in partitions "+partitionsToGetOffsetsFor.toString()+" " +
-										"Exceptions: "+exception);
-							}
-							// get valid offsets for these partitions and try again.
-							LOG.warn("The following partitions had an invalid offset: {}", partitionsToGetOffsetsFor);
-							getLastOffset(consumer, topic, partitionsToGetOffsetsFor, getInvalidOffsetBehavior(config));
-							LOG.warn("The new partition offsets are {}", partitionsToGetOffsetsFor);
-							continue; // jump back to create a new fetch request. The offset has not been touched.
-						} else {
-							// all partitions failed on an error
-							throw new IOException("Error while fetching from broker: " + exception);
-						}
-					}
-
-					int messagesInFetch = 0;
-					for (FetchPartition fp : partitions) {
-						final ByteBufferMessageSet messageSet = fetchResponse.messageSet(topic, fp.partition);
-						final int partition = fp.partition;
-						
-						for (MessageAndOffset msg : messageSet) {
-							if (running) {
-								messagesInFetch++;
-								if (msg.offset() < fp.nextOffsetToRead) {
-									// we have seen this message already
-									LOG.info("Skipping message with offset " + msg.offset()
-											+ " because we have seen messages until " + fp.nextOffsetToRead
-											+ " from partition " + fp.partition + " already");
-									continue;
-								}
-								
-								ByteBuffer payload = msg.message().payload();
-								byte[] valueByte = new byte[payload.remaining()];
-								payload.get(valueByte);
-								
-								final T value = valueDeserializer.deserialize(valueByte);
-								final long offset = msg.offset();
-										
-								synchronized (sourceContext.getCheckpointLock()) {
-									sourceContext.collect(value);
-									offsetsState[partition] = offset;
-								}
-								
-								// advance offset for the next request
-								fp.nextOffsetToRead = offset + 1;
-							}
-							else {
-								// no longer running
-								return;
-							}
-						}
-					}
-					LOG.debug("This fetch contained {} messages", messagesInFetch);
-				}
-			}
-			catch (Throwable t) {
-				// report to the main thread
-				owner.onErrorInFetchThread(t);
-			}
-			finally {
-				// end of run loop. close connection to consumer
-				if (consumer != null) {
-					// closing the consumer should not fail the program
-					try {
-						consumer.close();
-					}
-					catch (Throwable t) {
-						LOG.error("Error while closing the Kafka simple consumer", t);
-					}
-				}
-			}
-		}
-
-		/**
-		 * Cancels this fetch thread. The thread will release all resources and terminate.
-		 */
-		public void cancel() {
-			this.running = false;
-			
-			// interrupt whatever the consumer is doing
-			if (consumer != null) {
-				consumer.close();
-			}
-			
-			this.interrupt();
-		}
-
-		/**
-		 * Request latest offsets for a set of partitions, via a Kafka consumer.
-		 *
-		 * @param consumer The consumer connected to lead broker
-		 * @param topic The topic name
-		 * @param partitions The list of partitions we need offsets for
-		 * @param whichTime The type of time we are requesting. -1 and -2 are special constants (See OffsetRequest)
-		 */
-		private static void getLastOffset(SimpleConsumer consumer, String topic, List<FetchPartition> partitions, long whichTime) {
-
-			Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<>();
-			for (FetchPartition fp: partitions) {
-				TopicAndPartition topicAndPartition = new TopicAndPartition(topic, fp.partition);
-				requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(whichTime, 1));
-			}
-
-			kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
-			OffsetResponse response = consumer.getOffsetsBefore(request);
-
-			if (response.hasError()) {
-				String exception = "";
-				for (FetchPartition fp: partitions) {
-					short code;
-					if ( (code=response.errorCode(topic, fp.partition)) != ErrorMapping.NoError()) {
-						exception += "\nException for partition "+fp.partition+": "+ StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
-					}
-				}
-				throw new RuntimeException("Unable to get last offset for topic " + topic + " and partitions " + partitions
-						+ ". " + exception);
-			}
-
-			for (FetchPartition fp: partitions) {
-				// the resulting offset is the next offset we are going to read
-				// for not-yet-consumed partitions, it is 0.
-				fp.nextOffsetToRead = response.offsets(topic, fp.partition)[0];
-			}
-		}
-
-		private static long getInvalidOffsetBehavior(Properties config) {
-			long timeType;
-			if (config.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest").equals("latest")) {
-				timeType = OffsetRequest.LatestTime();
-			} else {
-				timeType = OffsetRequest.EarliestTime();
-			}
-			return timeType;
-		}
-	}
-	
-	private static class PartitionInfoFetcher extends Thread {
-
-		private final String topic;
-		private final Properties properties;
-		
-		private volatile List<PartitionInfo> result;
-		private volatile Throwable error;
-
-		
-		PartitionInfoFetcher(String topic, Properties properties) {
-			this.topic = topic;
-			this.properties = properties;
-		}
-
-		@Override
-		public void run() {
-			try {
-				result = FlinkKafkaConsumer.getPartitionsForTopic(topic, properties);
-			}
-			catch (Throwable t) {
-				this.error = t;
-			}
-		}
-		
-		public List<PartitionInfo> getPartitions() throws Exception {
-			try {
-				this.join();
-			}
-			catch (InterruptedException e) {
-				throw new Exception("Partition fetching was cancelled before completion");
-			}
-			
-			if (error != null) {
-				throw new Exception("Failed to fetch partitions for topic " + topic, error);
-			}
-			if (result != null) {
-				return result;
-			}
-			throw new Exception("Partition fetching failed");
-		}
-	}
-
-	private static class KillerWatchDog extends Thread {
-		
-		private final Thread toKill;
-		private final long timeout;
-
-		private KillerWatchDog(Thread toKill, long timeout) {
-			super("KillerWatchDog");
-			setDaemon(true);
-			
-			this.toKill = toKill;
-			this.timeout = timeout;
-		}
-
-		@SuppressWarnings("deprecation")
-		@Override
-		public void run() {
-			final long deadline = System.currentTimeMillis() + timeout;
-			long now;
-			
-			while (toKill.isAlive() && (now = System.currentTimeMillis()) < deadline) {
-				try {
-					toKill.join(deadline - now);
-				}
-				catch (InterruptedException e) {
-					// ignore here, our job is important!
-				}
-			}
-			
-			// this is harsh, but this watchdog is a last resort
-			if (toKill.isAlive()) {
-				toKill.stop();
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.java
deleted file mode 100644
index 2a82561..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/OffsetHandler.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.connectors.kafka.internals;
-
-
-import org.apache.kafka.common.TopicPartition;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-/**
- * The offset handler is responsible for locating the initial partition offsets 
- * where the source should start reading, as well as committing offsets from completed
- * checkpoints.
- */
-public interface OffsetHandler {
-
-	/**
-	 * Commits the given offset for the partitions. May commit the offsets to the Kafka broker,
-	 * or to ZooKeeper, based on its configured behavior.
-	 *
-	 * @param offsetsToCommit The offset to commit, per partition.
-	 */
-	void commit(Map<TopicPartition, Long> offsetsToCommit) throws Exception;
-
-	/**
-	 * Positions the given fetcher to the initial read offsets where the stream consumption
-	 * will start from.
-	 * 
-	 * @param partitions The partitions for which to seeks the fetcher to the beginning.
-	 * @param fetcher The fetcher that will pull data from Kafka and must be positioned.
-	 */
-	void seekFetcherToInitialOffsets(List<TopicPartition> partitions, Fetcher fetcher) throws Exception;
-
-	/**
-	 * Closes the offset handler, releasing all resources.
-	 * 
-	 * @throws IOException Thrown, if the closing fails.
-	 */
-	void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
deleted file mode 100644
index a38c3bd..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionerWrapper.java
+++ /dev/null
@@ -1,49 +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.connectors.kafka.internals;
-
-import kafka.producer.Partitioner;
-import kafka.utils.VerifiableProperties;
-
-/**
- * Hacky wrapper to send an object instance through a Properties - map.
- *
- * This works as follows:
- * The recommended way of creating a KafkaSink is specifying a classname for the partitioner.
- *
- * Otherwise (if the user gave a (serializable) class instance), we give Kafka the PartitionerWrapper class of Flink.
- * This is set in the key-value (java.util.Properties) map.
- * In addition to that, we use the Properties.put(Object, Object) to store the instance of the (serializable).
- * This is a hack because the put() method is called on the underlying Hashmap.
- *
- * This PartitionerWrapper is called with the Properties. From there, we extract the wrapped Partitioner instance.
- *
- * The serializable PartitionerWrapper is serialized into the Properties Hashmap and also deserialized from there.
- */
-public class PartitionerWrapper implements Partitioner {
-	public final static String SERIALIZED_WRAPPER_NAME = "flink.kafka.wrapper.serialized";
-
-	private Partitioner wrapped;
-	public PartitionerWrapper(VerifiableProperties properties) {
-		wrapped = (Partitioner) properties.props().get(SERIALIZED_WRAPPER_NAME);
-	}
-
-	@Override
-	public int partition(Object value, int numberOfPartitions) {
-		return wrapped.partition(value, numberOfPartitions);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
deleted file mode 100644
index 001b6cb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZooKeeperStringSerializer.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import org.I0Itec.zkclient.serialize.ZkSerializer;
-
-import java.nio.charset.Charset;
-
-/**
- * Simple ZooKeeper serializer for Strings.
- */
-public class ZooKeeperStringSerializer implements ZkSerializer {
-
-	private static final Charset CHARSET = Charset.forName("UTF-8");
-	
-	@Override
-	public byte[] serialize(Object data) {
-		if (data instanceof String) {
-			return ((String) data).getBytes(CHARSET);
-		}
-		else {
-			throw new IllegalArgumentException("ZooKeeperStringSerializer can only serialize strings.");
-		}
-	}
-
-	@Override
-	public Object deserialize(byte[] bytes) {
-		if (bytes == null) {
-			return null;
-		}
-		else {
-			return new String(bytes, CHARSET);
-		}
-	}
-}


[22/51] [partial] flink git commit: [FLINK-2877] Move Streaming API out of Staging package

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CollectingOutput.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CollectingOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CollectingOutput.java
deleted file mode 100644
index 3c1c24b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CollectingOutput.java
+++ /dev/null
@@ -1,80 +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.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class CollectingOutput<T> implements Output<StreamRecord<T>> {
-	
-	private final List<T> elements = new ArrayList<>();
-
-	private final int timeStampModulus;
-
-
-	public CollectingOutput() {
-		this.timeStampModulus = 0;
-	}
-	
-	public CollectingOutput(int timeStampModulus) {
-		this.timeStampModulus = timeStampModulus;
-	}
-
-	// ------------------------------------------------------------------------
-	
-	public List<T> getElements() {
-		return elements;
-	}
-	
-	public void waitForNElements(int n, long timeout) throws InterruptedException {
-		long deadline = System.currentTimeMillis() + timeout;
-		synchronized (elements) {
-			long now;
-			while (elements.size() < n && (now = System.currentTimeMillis()) < deadline) {
-				elements.wait(deadline - now);
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public void emitWatermark(Watermark mark) {
-		throw new UnsupportedOperationException("the output should not emit watermarks");
-	}
-
-	@Override
-	public void collect(StreamRecord<T> record) {
-		elements.add(record.getValue());
-		
-		if (timeStampModulus != 0 && record.getTimestamp() % timeStampModulus != 0) {
-			throw new IllegalArgumentException("Invalid timestamp");
-		}
-		synchronized (elements) {
-			elements.notifyAll();
-		}
-	}
-
-	@Override
-	public void close() {}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 39033cc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingNonKeyedWindowOperatorTest.java
+++ /dev/null
@@ -1,173 +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.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.ReduceAllWindowFunction;
-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 GlobalWindow.Serializer(),
-				new HeapWindowBuffer.Factory<Tuple2<String, Integer>>(),
-				new ReduceAllWindowFunction<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/06f6ac5d/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
deleted file mode 100644
index 1821308..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
+++ /dev/null
@@ -1,182 +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.ExecutionConfig;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-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.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.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
-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 EvictingWindowOperatorTest {
-
-	// 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;
-
-		EvictingWindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new EvictingWindowOperator<>(
-				GlobalWindows.create(),
-				new GlobalWindow.Serializer(),
-				new TupleKeySelector(),
-				BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()),
-				new HeapWindowBuffer.Factory<Tuple2<String, Integer>>(),
-				new ReduceWindowFunction<String, 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<>("key1", 1), initialTime + 20));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime + 999));
-
-		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));
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 2), Long.MAX_VALUE));
-
-		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
-
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key1", 1), initialTime + 10999));
-		testHarness.processElement(new StreamRecord<>(new Tuple2<>("key2", 1), initialTime + 1000));
-
-		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 4), 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.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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutIfAbsentTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutIfAbsentTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutIfAbsentTest.java
deleted file mode 100644
index c0b20a3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutIfAbsentTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class KeyMapPutIfAbsentTest {
-	
-	@Test
-	public void testPutIfAbsentUniqueKeysAndGrowth() {
-		try {
-			KeyMap<Integer, Integer> map = new KeyMap<>();
-			IntegerFactory factory = new IntegerFactory();
-			
-			final int numElements = 1000000;
-			
-			for (int i = 0; i < numElements; i++) {
-				factory.set(2 * i + 1);
-				map.putIfAbsent(i, factory);
-
-				assertEquals(i+1, map.size());
-				assertTrue(map.getCurrentTableCapacity() > map.size());
-				assertTrue(map.getCurrentTableCapacity() > map.getRehashThreshold());
-				assertTrue(map.size() <= map.getRehashThreshold());
-			}
-			
-			assertEquals(numElements, map.size());
-			assertEquals(numElements, map.traverseAndCountElements());
-			assertEquals(1 << 21, map.getCurrentTableCapacity());
-
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(2 * i + 1, map.get(i).intValue());
-			}
-			
-			for (int i = numElements - 1; i >= 0; i--) {
-				assertEquals(2 * i + 1, map.get(i).intValue());
-			}
-
-			assertEquals(numElements, map.size());
-			assertEquals(numElements, map.traverseAndCountElements());
-			assertEquals(1 << 21, map.getCurrentTableCapacity());
-			assertTrue(map.getLongestChainLength() <= 7);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testPutIfAbsentDuplicateKeysAndGrowth() {
-		try {
-			KeyMap<Integer, Integer> map = new KeyMap<>();
-			IntegerFactory factory = new IntegerFactory();
-			
-			final int numElements = 1000000;
-
-			for (int i = 0; i < numElements; i++) {
-				int val = 2 * i + 1;
-				factory.set(val);
-				Integer put = map.putIfAbsent(i, factory);
-				assertEquals(val, put.intValue());
-			}
-
-			for (int i = 0; i < numElements; i += 3) {
-				factory.set(2 * i);
-				Integer put = map.putIfAbsent(i, factory);
-				assertEquals(2 * i + 1, put.intValue());
-			}
-
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(2 * i + 1, map.get(i).intValue());
-			}
-
-			assertEquals(numElements, map.size());
-			assertEquals(numElements, map.traverseAndCountElements());
-			assertEquals(1 << 21, map.getCurrentTableCapacity());
-			assertTrue(map.getLongestChainLength() <= 7);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private static class IntegerFactory implements KeyMap.LazyFactory<Integer> {
-		
-		private Integer toCreate;
-		
-		public void set(Integer toCreate) {
-			this.toCreate = toCreate;
-		}
-
-		@Override
-		public Integer create() {
-			return toCreate;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutTest.java
deleted file mode 100644
index 09c44fe..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapPutTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.runtime.operators.windowing;
-
-import org.junit.Test;
-
-import java.util.BitSet;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class KeyMapPutTest {
-
-	@Test
-	public void testPutUniqueKeysAndGrowth() {
-		try {
-			KeyMap<Integer, Integer> map = new KeyMap<>();
-
-			final int numElements = 1000000;
-
-			for (int i = 0; i < numElements; i++) {
-				map.put(i, 2 * i + 1);
-
-				assertEquals(i+1, map.size());
-				assertTrue(map.getCurrentTableCapacity() > map.size());
-				assertTrue(map.getCurrentTableCapacity() > map.getRehashThreshold());
-				assertTrue(map.size() <= map.getRehashThreshold());
-			}
-
-			assertEquals(numElements, map.size());
-			assertEquals(numElements, map.traverseAndCountElements());
-			assertEquals(1 << 21, map.getCurrentTableCapacity());
-
-			for (int i = 0; i < numElements; i++) {
-				assertEquals(2 * i + 1, map.get(i).intValue());
-			}
-
-			for (int i = numElements - 1; i >= 0; i--) {
-				assertEquals(2 * i + 1, map.get(i).intValue());
-			}
-
-			BitSet bitset = new BitSet();
-			int numContained = 0;
-			for (KeyMap.Entry<Integer, Integer> entry : map) {
-				numContained++;
-				
-				assertEquals(entry.getKey() * 2 + 1, entry.getValue().intValue());
-				assertFalse(bitset.get(entry.getKey()));
-				bitset.set(entry.getKey());
-			}
-
-			assertEquals(numElements, numContained);
-			assertEquals(numElements, bitset.cardinality());
-			
-			
-			assertEquals(numElements, map.size());
-			assertEquals(numElements, map.traverseAndCountElements());
-			assertEquals(1 << 21, map.getCurrentTableCapacity());
-			assertTrue(map.getLongestChainLength() <= 7);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testPutDuplicateKeysAndGrowth() {
-		try {
-			final KeyMap<Integer, Integer> map = new KeyMap<>();
-			final int numElements = 1000000;
-
-			for (int i = 0; i < numElements; i++) {
-				Integer put = map.put(i, 2*i+1);
-				assertNull(put);
-			}
-
-			for (int i = 0; i < numElements; i += 3) {
-				Integer put = map.put(i, 2*i);
-				assertNotNull(put);
-				assertEquals(2*i+1, put.intValue());
-			}
-
-			for (int i = 0; i < numElements; i++) {
-				int expected = (i % 3 == 0) ? (2*i) : (2*i+1);
-				assertEquals(expected, map.get(i).intValue());
-			}
-			
-			assertEquals(numElements, map.size());
-			assertEquals(numElements, map.traverseAndCountElements());
-			assertEquals(1 << 21, map.getCurrentTableCapacity());
-			assertTrue(map.getLongestChainLength() <= 7);
-
-			
-			BitSet bitset = new BitSet();
-			int numContained = 0;
-			for (KeyMap.Entry<Integer, Integer> entry : map) {
-				numContained++;
-
-				int key = entry.getKey();
-				int expected = key % 3 == 0 ? (2*key) : (2*key+1);
-
-				assertEquals(expected, entry.getValue().intValue());
-				assertFalse(bitset.get(key));
-				bitset.set(key);
-			}
-
-			assertEquals(numElements, numContained);
-			assertEquals(numElements, bitset.cardinality());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapTest.java
deleted file mode 100644
index 49310df..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/KeyMapTest.java
+++ /dev/null
@@ -1,344 +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.junit.Test;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Random;
-
-import static org.junit.Assert.*;
-
-public class KeyMapTest {
-	
-	@Test
-	public void testInitialSizeComputation() {
-		try {
-			KeyMap<String, String> map;
-
-			map = new KeyMap<>();
-			assertEquals(64, map.getCurrentTableCapacity());
-			assertEquals(6, map.getLog2TableCapacity());
-			assertEquals(24, map.getShift());
-			assertEquals(48, map.getRehashThreshold());
-			
-			map = new KeyMap<>(0);
-			assertEquals(64, map.getCurrentTableCapacity());
-			assertEquals(6, map.getLog2TableCapacity());
-			assertEquals(24, map.getShift());
-			assertEquals(48, map.getRehashThreshold());
-
-			map = new KeyMap<>(1);
-			assertEquals(64, map.getCurrentTableCapacity());
-			assertEquals(6, map.getLog2TableCapacity());
-			assertEquals(24, map.getShift());
-			assertEquals(48, map.getRehashThreshold());
-
-			map = new KeyMap<>(9);
-			assertEquals(64, map.getCurrentTableCapacity());
-			assertEquals(6, map.getLog2TableCapacity());
-			assertEquals(24, map.getShift());
-			assertEquals(48, map.getRehashThreshold());
-
-			map = new KeyMap<>(63);
-			assertEquals(64, map.getCurrentTableCapacity());
-			assertEquals(6, map.getLog2TableCapacity());
-			assertEquals(24, map.getShift());
-			assertEquals(48, map.getRehashThreshold());
-
-			map = new KeyMap<>(64);
-			assertEquals(128, map.getCurrentTableCapacity());
-			assertEquals(7, map.getLog2TableCapacity());
-			assertEquals(23, map.getShift());
-			assertEquals(96, map.getRehashThreshold());
-
-			map = new KeyMap<>(500);
-			assertEquals(512, map.getCurrentTableCapacity());
-			assertEquals(9, map.getLog2TableCapacity());
-			assertEquals(21, map.getShift());
-			assertEquals(384, map.getRehashThreshold());
-
-			map = new KeyMap<>(127);
-			assertEquals(128, map.getCurrentTableCapacity());
-			assertEquals(7, map.getLog2TableCapacity());
-			assertEquals(23, map.getShift());
-			assertEquals(96, map.getRehashThreshold());
-			
-			// no negative number of elements
-			try {
-				new KeyMap<>(-1);
-				fail("should fail with an exception");
-			}
-			catch (IllegalArgumentException e) {
-				// expected
-			}
-			
-			// check integer overflow
-			try {
-				map = new KeyMap<>(0x65715522);
-
-				final int maxCap = Integer.highestOneBit(Integer.MAX_VALUE);
-				assertEquals(Integer.highestOneBit(Integer.MAX_VALUE), map.getCurrentTableCapacity());
-				assertEquals(30, map.getLog2TableCapacity());
-				assertEquals(0, map.getShift());
-				assertEquals(maxCap / 4 * 3, map.getRehashThreshold());
-			}
-			catch (OutOfMemoryError e) {
-				// this may indeed happen in small test setups. we tolerate this in this test
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testPutAndGetRandom() {
-		try {
-			final KeyMap<Integer, Integer> map = new KeyMap<>();
-			final Random rnd = new Random();
-			
-			final long seed = rnd.nextLong();
-			final int numElements = 10000;
-			
-			final HashMap<Integer, Integer> groundTruth = new HashMap<>();
-			
-			rnd.setSeed(seed);
-			for (int i = 0; i < numElements; i++) {
-				Integer key = rnd.nextInt();
-				Integer value = rnd.nextInt();
-				
-				if (rnd.nextBoolean()) {
-					groundTruth.put(key, value);
-					map.put(key, value);
-				}
-			}
-
-			rnd.setSeed(seed);
-			for (int i = 0; i < numElements; i++) {
-				Integer key = rnd.nextInt();
-
-				// skip these, evaluating it is tricky due to duplicates
-				rnd.nextInt();
-				rnd.nextBoolean();
-				
-				Integer expected = groundTruth.get(key);
-				if (expected == null) {
-					assertNull(map.get(key));
-				}
-				else {
-					Integer contained = map.get(key);
-					assertNotNull(contained);
-					assertEquals(expected, contained);
-				}
-			}
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testConjunctTraversal() {
-		try {
-			final Random rootRnd = new Random(654685486325439L);
-			
-			final int numMaps = 7;
-			final int numKeys = 1000000;
-
-			// ------ create a set of maps ------
-			@SuppressWarnings("unchecked")
-			final KeyMap<Integer, Integer>[] maps = (KeyMap<Integer, Integer>[]) new KeyMap<?, ?>[numMaps];
-			for (int i = 0; i < numMaps; i++) {
-				maps[i] = new KeyMap<>();
-			}
-			
-			// ------ prepare probabilities for maps ------
-			final double[] probabilities = new double[numMaps];
-			final double[] probabilitiesTemp = new double[numMaps];
-			{
-				probabilities[0] = 0.5;
-				double remainingProb = 1.0 - probabilities[0];
-				for (int i = 1; i < numMaps - 1; i++) {
-					remainingProb /= 2;
-					probabilities[i] = remainingProb;
-				}
-
-				// compensate for rounding errors
-				probabilities[numMaps - 1] = remainingProb;
-			}
-			
-			// ------ generate random elements ------
-			final long probSeed = rootRnd.nextLong();
-			final long keySeed = rootRnd.nextLong();
-			
-			final Random probRnd = new Random(probSeed);
-			final Random keyRnd = new Random(keySeed);
-			
-			final int maxStride = Integer.MAX_VALUE / numKeys;
-			
-			int totalNumElements = 0;
-			int nextKeyValue = 1;
-			
-			for (int i = 0; i < numKeys; i++) {
-				int numCopies = (nextKeyValue % 3) + 1;
-				System.arraycopy(probabilities, 0, probabilitiesTemp, 0, numMaps);
-				
-				double totalProb = 1.0;
-				for (int copy = 0; copy < numCopies; copy++) {
-					int pos = drawPosProportionally(probabilitiesTemp, totalProb, probRnd);
-					totalProb -= probabilitiesTemp[pos];
-					probabilitiesTemp[pos] = 0.0;
-					
-					Integer boxed = nextKeyValue;
-					Integer previous = maps[pos].put(boxed, boxed);
-					assertNull("Test problem - test does not assign unique maps", previous);
-				}
-				
-				totalNumElements += numCopies;
-				nextKeyValue += keyRnd.nextInt(maxStride) + 1;
-			}
-			
-			
-			// check that all maps contain the total number of elements
-			int numContained = 0;
-			for (KeyMap<?, ?> map : maps) {
-				numContained += map.size();
-			}
-			assertEquals(totalNumElements, numContained);
-
-			// ------ check that all elements can be found in the maps ------
-			keyRnd.setSeed(keySeed);
-			
-			numContained = 0;
-			nextKeyValue = 1;
-			for (int i = 0; i < numKeys; i++) {
-				int numCopiesExpected = (nextKeyValue % 3) + 1;
-				int numCopiesContained = 0;
-				
-				for (KeyMap<Integer, Integer> map : maps) {
-					Integer val = map.get(nextKeyValue);
-					if (val != null) {
-						assertEquals(nextKeyValue, val.intValue());
-						numCopiesContained++;
-					}
-				}
-				
-				assertEquals(numCopiesExpected, numCopiesContained);
-				numContained += numCopiesContained;
-				
-				nextKeyValue += keyRnd.nextInt(maxStride) + 1;
-			}
-			assertEquals(totalNumElements, numContained);
-
-			// ------ make a traversal over all keys and validate the keys in the traversal ------
-			final int[] keysStartedAndFinished = { 0, 0 };
-			KeyMap.TraversalEvaluator<Integer, Integer> traversal = new KeyMap.TraversalEvaluator<Integer, Integer>() {
-
-				private int key;
-				private int valueCount;
-				
-				@Override
-				public void startNewKey(Integer key) {
-					this.key = key;
-					this.valueCount = 0;
-					
-					keysStartedAndFinished[0]++;
-				}
-
-				@Override
-				public void nextValue(Integer value) {
-					assertEquals(this.key, value.intValue());
-					this.valueCount++;
-				}
-
-				@Override
-				public void keyDone() {
-					int expected = (key % 3) + 1;
-					if (expected != valueCount) {
-						fail("Wrong count for key " + key + " ; expected=" + expected + " , count=" + valueCount);
-					}
-					
-					keysStartedAndFinished[1]++;
-				}
-			};
-			
-			KeyMap.traverseMaps(shuffleArray(maps, rootRnd), traversal, 17);
-			
-			assertEquals(numKeys, keysStartedAndFinished[0]);
-			assertEquals(numKeys, keysStartedAndFinished[1]);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	@Test
-	public void testSizeComparator() {
-		try {
-			KeyMap<String, String> map1 = new KeyMap<>(5);
-			KeyMap<String, String> map2 = new KeyMap<>(80);
-			
-			assertTrue(map1.getCurrentTableCapacity() < map2.getCurrentTableCapacity());
-			
-			assertTrue(KeyMap.CapacityDescendingComparator.INSTANCE.compare(map1, map1) == 0);
-			assertTrue(KeyMap.CapacityDescendingComparator.INSTANCE.compare(map2, map2) == 0);
-			assertTrue(KeyMap.CapacityDescendingComparator.INSTANCE.compare(map1, map2) > 0);
-			assertTrue(KeyMap.CapacityDescendingComparator.INSTANCE.compare(map2, map1) < 0);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-
-	private static int drawPosProportionally(double[] array, double totalProbability, Random rnd) {
-		double val = rnd.nextDouble() * totalProbability;
-		
-		double accum = 0;
-		for (int i = 0; i < array.length; i++) {
-			accum += array[i];
-			if (val <= accum && array[i] > 0.0) {
-				return i;
-			}
-		}
-		
-		// in case of rounding errors
-		return array.length - 1;
-	}
-	
-	private static <E> E[] shuffleArray(E[] array, Random rnd) {
-		E[] target = Arrays.copyOf(array, array.length);
-		
-		for (int i = target.length - 1; i > 0; i--) {
-			int swapPos = rnd.nextInt(i + 1);
-			E temp = target[i];
-			target[i] = target[swapPos];
-			target[swapPos] = temp;
-		}
-		
-		return target;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 02e032a..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperatorTest.java
+++ /dev/null
@@ -1,429 +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.ExecutionConfig;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-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.ReduceAllWindowFunction;
-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.time.Time;
-import org.apache.flink.streaming.api.windowing.triggers.ContinuousEventTimeTrigger;
-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.EventTimeTrigger;
-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.TimeUnit;
-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 = 3;
-		final int WINDOW_SLIDE = 1;
-
-		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
-				SlidingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS), Time.of(WINDOW_SLIDE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				windowBufferFactory,
-				new ReduceAllWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
-				EventTimeTrigger.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 = 3;
-
-		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new NonKeyedWindowOperator<>(
-				TumblingTimeWindows.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
-				new TimeWindow.Serializer(),
-				windowBufferFactory,
-				new ReduceAllWindowFunction<TimeWindow, Tuple2<String, Integer>>(new SumReducer()),
-				EventTimeTrigger.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 = 3;
-
-		NonKeyedWindowOperator<Tuple2<String, Integer>, Tuple2<String, Integer>, GlobalWindow> operator = new NonKeyedWindowOperator<>(
-				GlobalWindows.create(),
-				new GlobalWindow.Serializer(),
-				windowBufferFactory,
-				new ReduceAllWindowFunction<GlobalWindow, Tuple2<String, Integer>>(new SumReducer()),
-				ContinuousEventTimeTrigger.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)));
-
-		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(),
-				new GlobalWindow.Serializer(),
-				windowBufferFactory,
-				new ReduceAllWindowFunction<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;
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/06f6ac5d/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
deleted file mode 100644
index 76c6f20..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
+++ /dev/null
@@ -1,143 +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.datastream.WindowedStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
-import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
-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.Ignore;
-import org.junit.Test;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * These tests verify that the api calls on
- * {@link WindowedStream} 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))
-				.reduce(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))
-				.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(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);
-	}
-
-	/**
-	 * 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))
-				.reduce(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))
-				.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
-					private static final long serialVersionUID = 1L;
-
-					@Override
-					public void apply(
-							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/06f6ac5d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
deleted file mode 100644
index fb7142b..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
+++ /dev/null
@@ -1,191 +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 com.google.common.collect.Lists;
-import org.apache.flink.api.common.functions.FoldFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Tests for Folds over windows. These also test whether OutputTypeConfigurable functions
- * work for windows, because FoldWindowFunction is OutputTypeConfigurable.
- */
-public class WindowFoldITCase extends StreamingMultipleProgramsTestBase {
-
-	private static List<String> testResults;
-
-	@Test
-	public void testFoldWindow() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple2.of("a", 0));
-				ctx.collect(Tuple2.of("a", 1));
-				ctx.collect(Tuple2.of("a", 2));
-
-				ctx.collect(Tuple2.of("b", 3));
-				ctx.collect(Tuple2.of("b", 4));
-				ctx.collect(Tuple2.of("b", 5));
-
-				ctx.collect(Tuple2.of("a", 6));
-				ctx.collect(Tuple2.of("a", 7));
-				ctx.collect(Tuple2.of("a", 8));
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).assignTimestamps(new Tuple2TimestampExtractor());
-
-		source1
-				.keyBy(0)
-				.window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.fold(Tuple2.of("R:", 0), new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
-					@Override
-					public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator,
-							Tuple2<String, Integer> value) throws Exception {
-						accumulator.f0 += value.f0;
-						accumulator.f1 += value.f1;
-						return accumulator;
-					}
-				})
-				.addSink(new SinkFunction<Tuple2<String, Integer>>() {
-					@Override
-					public void invoke(Tuple2<String, Integer> value) throws Exception {
-						testResults.add(value.toString());
-					}
-				});
-
-		env.execute("Fold Window Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"(R:aaa,3)",
-				"(R:aaa,21)",
-				"(R:bbb,12)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	@Test
-	public void testFoldAllWindow() throws Exception {
-
-		testResults = Lists.newArrayList();
-
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(1);
-
-		DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
-			private static final long serialVersionUID = 1L;
-
-			@Override
-			public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
-				ctx.collect(Tuple2.of("a", 0));
-				ctx.collect(Tuple2.of("a", 1));
-				ctx.collect(Tuple2.of("a", 2));
-
-				ctx.collect(Tuple2.of("b", 3));
-				ctx.collect(Tuple2.of("a", 3));
-				ctx.collect(Tuple2.of("b", 4));
-				ctx.collect(Tuple2.of("a", 4));
-				ctx.collect(Tuple2.of("b", 5));
-				ctx.collect(Tuple2.of("a", 5));
-
-			}
-
-			@Override
-			public void cancel() {
-			}
-		}).assignTimestamps(new Tuple2TimestampExtractor());
-
-		source1
-				.windowAll(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
-				.fold(Tuple2.of("R:", 0), new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
-					@Override
-					public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator,
-							Tuple2<String, Integer> value) throws Exception {
-						accumulator.f0 += value.f0;
-						accumulator.f1 += value.f1;
-						return accumulator;
-					}
-				})
-				.addSink(new SinkFunction<Tuple2<String, Integer>>() {
-					@Override
-					public void invoke(Tuple2<String, Integer> value) throws Exception {
-						testResults.add(value.toString());
-					}
-				});
-
-		env.execute("Fold All-Window Test");
-
-		List<String> expectedResult = Lists.newArrayList(
-				"(R:aaa,3)",
-				"(R:bababa,24)");
-
-		Collections.sort(expectedResult);
-		Collections.sort(testResults);
-
-		Assert.assertEquals(expectedResult, testResults);
-	}
-
-	private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
-			return element.f1;
-		}
-
-		@Override
-		public long extractWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
-			return element.f1 - 1;
-		}
-
-		@Override
-		public long getCurrentWatermark() {
-			return Long.MIN_VALUE;
-		}
-	}
-}