You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2016/10/05 22:16:51 UTC

[01/17] flink git commit: [FLINK-4737] [core] Ensure that Flink and its Hadoop dependency pull the same version of 'commons-compress'

Repository: flink
Updated Branches:
  refs/heads/master 10a42f951 -> 417c5a4b4


[FLINK-4737] [core] Ensure that Flink and its Hadoop dependency pull the same version of 'commons-compress'


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

Branch: refs/heads/master
Commit: 7a76328d4ec984aaa93d3e3181873254a6b85eed
Parents: 81aec41
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Oct 4 18:18:09 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:13 2016 +0200

----------------------------------------------------------------------
 flink-core/pom.xml                              | 31 +++++++++-----------
 .../flink-shaded-hadoop2/pom.xml                |  8 +++++
 pom.xml                                         |  6 ++++
 3 files changed, 28 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7a76328d/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index 40e5a2e..865a253 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -53,6 +53,20 @@ under the License.
 			<!-- managed version -->
 		</dependency>
 
+		<!-- The common collections are needed for some hash tables used in the collection execution -->
+		<dependency>
+			<groupId>commons-collections</groupId>
+			<artifactId>commons-collections</artifactId>
+			<!-- managed version -->
+		</dependency>
+
+		<!-- Commons compression, for additional decompressors -->
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-compress</artifactId>
+			<!-- managed version -->
+		</dependency>
+
 		<!-- Avro is needed for the interoperability with Avro types for serialization -->
 		<dependency>
 			<groupId>org.apache.avro</groupId>
@@ -63,26 +77,9 @@ under the License.
 					<groupId>org.xerial.snappy</groupId>
 					<artifactId>snappy-java</artifactId>
 				</exclusion>
-				<exclusion>
-					<groupId>org.apache.commons</groupId>
-					<artifactId>commons-compress</artifactId>
-				</exclusion>
 			</exclusions>
 		</dependency>
 
-		<!-- The common collections are needed for some hash tables used in the collection execution -->
-		<dependency>
-			<groupId>commons-collections</groupId>
-			<artifactId>commons-collections</artifactId>
-		</dependency>
-
-		<!-- Commons compression, for additional decompressors -->
-		<dependency>
-			<groupId>org.apache.commons</groupId>
-			<artifactId>commons-compress</artifactId>
-			<version>1.4</version>
-		</dependency>
-
 		<!-- test dependencies -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/7a76328d/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
index 8a7b101..7df2578 100644
--- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
@@ -35,6 +35,14 @@ under the License.
 	<packaging>jar</packaging>
 
 	<dependencies>
+
+		<!-- explicitly include commons-compress to force Flink's managed version -->
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-compress</artifactId>
+			<!-- managed version -->
+		</dependency>
+
 		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-common</artifactId>

http://git-wip-us.apache.org/repos/asf/flink/blob/7a76328d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d127ccb..330debe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -278,6 +278,12 @@ under the License.
 				<version>3.5</version>
 			</dependency>
 
+			<dependency>
+				<groupId>org.apache.commons</groupId>
+				<artifactId>commons-compress</artifactId>
+				<version>1.4.1</version>
+			</dependency>
+
 			<!-- Managed dependency required for HBase in flink-hbase -->
 			<dependency>
 				<groupId>org.javassist</groupId>


[11/17] flink git commit: [FLINK-4739] [elasticsearch connector] Adding packaging details for the Elasticsearch connector

Posted by se...@apache.org.
[FLINK-4739] [elasticsearch connector] Adding packaging details for the Elasticsearch connector

When an uber-jar containing an Elasticsearch sink is executed, an
IllegalArgumentException may occur, which is caused by conflicting files of
Elasticsearch and it's dependencies in META-INF/services.

This commit adds further clarification to the documentation on how to build a
porper uber-jar that can be properly executed by adapting the pom file.

This closes #2591


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

Branch: refs/heads/master
Commit: beb31fc1c5a6f5db060e5a4e1d64eacb70632900
Parents: 8000b64
Author: Steffen Hausmann <st...@hausmann-family.de>
Authored: Mon Oct 3 22:32:01 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:14 2016 +0200

----------------------------------------------------------------------
 docs/dev/connectors/elasticsearch2.md | 32 ++++++++++++++++++++++++++++++
 1 file changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/beb31fc1/docs/dev/connectors/elasticsearch2.md
----------------------------------------------------------------------
diff --git a/docs/dev/connectors/elasticsearch2.md b/docs/dev/connectors/elasticsearch2.md
index 8eed690..5f4267e 100644
--- a/docs/dev/connectors/elasticsearch2.md
+++ b/docs/dev/connectors/elasticsearch2.md
@@ -139,3 +139,35 @@ This now provides a list of Elasticsearch Nodes
 to which the sink should connect via a `TransportClient`.
 
 More information about Elasticsearch can be found [here](https://elastic.co).
+
+
+#### Packaging the Elasticsearch Connector into an Uber-jar
+
+For the execution of your Flink program,
+it is recommended to build a so-called uber-jar (executable jar) containing all your dependencies
+(see [here]({{site.baseurl}}/dev/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution) for further information).
+
+However,
+when an uber-jar containing an Elasticsearch sink is executed,
+an `IllegalArgumentException` may occur,
+which is caused by conflicting files of Elasticsearch and it's dependencies
+in `META-INF/services`:
+
+```
+IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist.  You need to add the corresponding JAR file supporting this SPI to your classpath.  The current classpath supports the following names: [es090, completion090, XBloomFilter]]
+```
+
+If the uber-jar is build by means of maven,
+this issue can be avoided by adding the following bits to the pom file:
+
+```
+<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+    <resource>META-INF/services/org.apache.lucene.codecs.Codec</resource>
+</transformer>
+<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+    <resource>META-INF/services/org.apache.lucene.codecs.DocValuesFormat</resource>
+</transformer>
+<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+   <resource>META-INF/services/org.apache.lucene.codecs.PostingsFormat</resource>
+</transformer>
+```


[14/17] flink git commit: [hotfix] [streaming api] Remove obsolete and unused InputTypeSerializer from WindowOperator

Posted by se...@apache.org.
[hotfix] [streaming api] Remove obsolete and unused InputTypeSerializer from WindowOperator


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

Branch: refs/heads/master
Commit: 1cd8d4f418a707790c091fed2428627eae9da423
Parents: 47e4977
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Oct 4 23:49:54 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 20:04:34 2016 +0200

----------------------------------------------------------------------
 .../operators/windowing/WindowOperator.java     | 20 +------
 .../windowing/EvictingWindowOperatorTest.java   |  6 ---
 .../operators/windowing/WindowOperatorTest.java | 56 +-------------------
 .../streaming/util/WindowingTestHarness.java    |  2 -
 4 files changed, 3 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1cd8d4f4/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index de316e7..c5f1ca2 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -20,7 +20,6 @@ package org.apache.flink.streaming.runtime.operators.windowing;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.AppendingState;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
@@ -34,7 +33,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.core.fs.FSDataInputStream;
@@ -98,7 +96,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 @Internal
 public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	extends AbstractUdfStreamOperator<OUT, InternalWindowFunction<ACC, OUT, K, W>>
-	implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
+	implements OneInputStreamOperator<IN, OUT>, Triggerable {
 
 	private static final long serialVersionUID = 1L;
 
@@ -115,12 +113,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	protected final StateDescriptor<? extends AppendingState<IN, ACC>, ?> windowStateDescriptor;
 
 	/**
-	 * This is used to copy the incoming element because it can be put into several window
-	 * buffers.
-	 */
-	protected TypeSerializer<IN> inputSerializer;
-
-	/**
 	 * For serializing the key in checkpoints.
 	 */
 	protected final TypeSerializer<K> keySerializer;
@@ -211,21 +203,11 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	}
 
 	@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.");
-		}
-
 		// these could already be initialized from restoreState()
 		if (watermarkTimers == null) {
 			watermarkTimers = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/1cd8d4f4/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
index 681a334..8f3af15 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java
@@ -82,8 +82,6 @@ public class EvictingWindowOperatorTest {
 				CountEvictor.of(WINDOW_SIZE),
 				0);
 
-		operator.setInputType(inputType, new ExecutionConfig());
-
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
@@ -152,8 +150,6 @@ public class EvictingWindowOperatorTest {
 			CountEvictor.of(WINDOW_SIZE),
 			0);
 
-		operator.setInputType(inputType, new ExecutionConfig());
-
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
@@ -221,8 +217,6 @@ public class EvictingWindowOperatorTest {
 			CountEvictor.of(WINDOW_SIZE),
 			0);
 
-		operator.setInputType(inputType, new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1cd8d4f4/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index e98bc91..cda6524 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -180,8 +180,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				0);
 
-		operator.setInputType(inputType, new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
@@ -216,8 +214,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				0);
 
-		operator.setInputType(inputType, new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
@@ -317,8 +313,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
@@ -351,8 +345,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
@@ -388,8 +380,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
@@ -462,8 +452,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
@@ -538,8 +526,6 @@ public class WindowOperatorTest extends TestLogger {
 				PurgingTrigger.of(CountTrigger.of(4)),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 		
@@ -643,8 +629,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 		
@@ -703,8 +687,6 @@ public class WindowOperatorTest extends TestLogger {
 				ContinuousEventTimeTrigger.of(Time.of(WINDOW_SIZE, TimeUnit.SECONDS)),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -793,9 +775,6 @@ public class WindowOperatorTest extends TestLogger {
 				PurgingTrigger.of(CountTrigger.of(WINDOW_SIZE)),
 				0);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(
-				"Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -839,9 +818,6 @@ public class WindowOperatorTest extends TestLogger {
 
 		testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse(
-				"Tuple2<String, Integer>"), new ExecutionConfig());
-
 		testHarness.setup();
 		testHarness.restore(snapshot);
 		testHarness.open();
@@ -876,7 +852,6 @@ public class WindowOperatorTest extends TestLogger {
 		final int WINDOW_SLIDE = 1;
 
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
-		TestTimeServiceProvider timer = new TestTimeServiceProvider();
 
 		TestTimeServiceProvider timer = new TestTimeServiceProvider();
 
@@ -957,7 +932,6 @@ public class WindowOperatorTest extends TestLogger {
 				ProcessingTimeTrigger.create(), 0);
 
 		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
@@ -1017,7 +991,6 @@ public class WindowOperatorTest extends TestLogger {
 				ProcessingTimeTrigger.create(), 0);
 
 		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
@@ -1090,7 +1063,6 @@ public class WindowOperatorTest extends TestLogger {
 				ProcessingTimeTrigger.create(), 0);
 
 		TestTimeServiceProvider testTimeProvider = new TestTimeServiceProvider();
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 				new KeyedOneInputStreamOperatorTestHarness<>(operator, new ExecutionConfig(), testTimeProvider, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
@@ -1159,8 +1131,7 @@ public class WindowOperatorTest extends TestLogger {
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
-
-		operator.setInputType(inputType, new ExecutionConfig());
+		
 		testHarness.open();
 		
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1220,8 +1191,7 @@ public class WindowOperatorTest extends TestLogger {
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
-
-		operator.setInputType(inputType, new ExecutionConfig());
+		
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1288,7 +1258,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1349,7 +1318,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 		
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -1422,8 +1390,6 @@ public class WindowOperatorTest extends TestLogger {
 				PurgingTrigger.of(EventTimeTrigger.create()),
 				LATENESS);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -1514,8 +1480,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				LATENESS);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -1600,8 +1564,6 @@ public class WindowOperatorTest extends TestLogger {
 				PurgingTrigger.of(EventTimeTrigger.create()),
 				LATENESS);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -1686,8 +1648,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				LATENESS);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -1781,8 +1741,6 @@ public class WindowOperatorTest extends TestLogger {
 				PurgingTrigger.of(EventTimeTrigger.create()),
 				LATENESS);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -1868,8 +1826,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				LATENESS);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -1958,7 +1914,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, String> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2013,7 +1968,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2060,7 +2014,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2116,7 +2069,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2162,7 +2114,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();
@@ -2204,8 +2155,6 @@ public class WindowOperatorTest extends TestLogger {
 				EventTimeTrigger.create(),
 				LATENESS);
 
-		operator.setInputType(TypeInfoParser.<Tuple2<String, Integer>>parse("Tuple2<String, Integer>"), new ExecutionConfig());
-
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple3<String, Long, Long>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
@@ -2261,7 +2210,6 @@ public class WindowOperatorTest extends TestLogger {
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
 			new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
 
-		operator.setInputType(inputType, new ExecutionConfig());
 		testHarness.open();
 
 		ConcurrentLinkedQueue<Object> expected = new ConcurrentLinkedQueue<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/1cd8d4f4/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java
index d47136c..ab8b70f 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/WindowingTestHarness.java
@@ -80,8 +80,6 @@ public class WindowingTestHarness<K, IN, W extends Window> {
 				trigger,
 				allowedLateness);
 
-		operator.setInputType(inputType, executionConfig);
-
 		timeServiceProvider = new TestTimeServiceProvider();
 		testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, executionConfig, timeServiceProvider, keySelector, keyType);
 	}


[04/17] flink git commit: [FLINK-4709] [core] Fix resource leak in InputStreamFSInputWrapper

Posted by se...@apache.org.
[FLINK-4709] [core] Fix resource leak in InputStreamFSInputWrapper

This closes #2581


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

Branch: refs/heads/master
Commit: 28b37ef97d803855e884900d1a983813ea2ac99d
Parents: 9206b48
Author: Holger Frydrych <ho...@tngtech.com>
Authored: Mon Oct 3 14:34:19 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:13 2016 +0200

----------------------------------------------------------------------
 .../common/io/InputStreamFSInputWrapper.java    |  5 +++
 .../io/InputStreamFSInputWrapperTest.java       | 38 ++++++++++++++++++++
 2 files changed, 43 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/28b37ef9/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java b/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java
index cfd94bc..f7db680 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java
@@ -42,6 +42,11 @@ public class InputStreamFSInputWrapper extends FSDataInputStream {
 	}
 
 	@Override
+	public void close() throws IOException {
+		this.inStream.close();
+	}
+
+	@Override
 	public void seek(long desired) throws IOException {
 		if (desired < this.pos) {
 			throw new IllegalArgumentException("Wrapped InputStream: cannot search backwards.");

http://git-wip-us.apache.org/repos/asf/flink/blob/28b37ef9/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java
new file mode 100644
index 0000000..8fcd231
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.io;
+
+import java.io.InputStream;
+
+import org.junit.Test;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class InputStreamFSInputWrapperTest {
+
+	@Test
+	public void testClose() throws Exception {
+		InputStream mockedInputStream = mock(InputStream.class);
+		InputStreamFSInputWrapper wrapper = new InputStreamFSInputWrapper(mockedInputStream);
+		wrapper.close();
+		verify(mockedInputStream).close();
+	}
+
+}


[02/17] flink git commit: [hotfix] Various code cleanups around time service and asynchronous exceptions

Posted by se...@apache.org.
[hotfix] Various code cleanups around time service and asynchronous exceptions

  - DefaultTimeServiceProvider now owns scheduled executor
  - Enforce that an asynchronous exception handler is always set


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

Branch: refs/heads/master
Commit: 954ef08f374d7e7c1f2b469201b1ea05c6376b33
Parents: 8ff451b
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Oct 4 16:15:05 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:13 2016 +0200

----------------------------------------------------------------------
 .../AbstractFetcherTimestampsTest.java          | 122 +++++++++++--------
 .../kafka/testutils/MockRuntimeContext.java     |  40 +++---
 .../api/operators/StreamSourceContexts.java     |   6 +-
 .../runtime/io/StreamInputProcessor.java        |   4 +-
 .../runtime/tasks/AsyncExceptionHandler.java    |   1 +
 .../runtime/tasks/AsynchronousException.java    |  11 +-
 .../tasks/DefaultTimeServiceProvider.java       |  57 +++++----
 .../runtime/tasks/OneInputStreamTask.java       |   2 +-
 .../streaming/runtime/tasks/StreamTask.java     |  18 +--
 .../runtime/tasks/TestTimeServiceProvider.java  |   2 +-
 .../runtime/tasks/TwoInputStreamTask.java       |   2 +-
 .../runtime/operators/TimeProviderTest.java     |  45 +++++--
 ...AlignedProcessingTimeWindowOperatorTest.java |  84 ++++++++-----
 ...AlignedProcessingTimeWindowOperatorTest.java | 101 +++++++++------
 .../operators/windowing/NoOpTimerService.java   |  49 ++++++++
 .../util/OneInputStreamOperatorTestHarness.java |   6 +-
 16 files changed, 347 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
index 8c68fbe..c3ba7b7 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
@@ -25,7 +25,10 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceCont
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
+import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
+import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
 import org.apache.flink.util.SerializedValue;
 
 import org.junit.Test;
@@ -34,6 +37,7 @@ import javax.annotation.Nullable;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
 
 import static org.junit.Assert.*;
 
@@ -110,6 +114,7 @@ public class AbstractFetcherTimestampsTest {
 	
 	@Test
 	public void testPeriodicWatermarks() throws Exception {
+
 		ExecutionConfig config = new ExecutionConfig();
 		config.setAutoWatermarkInterval(10);
 		
@@ -120,61 +125,70 @@ public class AbstractFetcherTimestampsTest {
 
 		TestSourceContext<Long> sourceContext = new TestSourceContext<>();
 
-		TestFetcher<Long> fetcher = new TestFetcher<>(
-				sourceContext, originalPartitions,
-				new SerializedValue<AssignerWithPeriodicWatermarks<Long>>(new PeriodicTestExtractor()),
-				null, new MockRuntimeContext(17, 3, config, sourceContext.getCheckpointLock()));
-
-		final KafkaTopicPartitionState<Object> part1 = fetcher.subscribedPartitions()[0];
-		final KafkaTopicPartitionState<Object> part2 = fetcher.subscribedPartitions()[1];
-		final KafkaTopicPartitionState<Object> part3 = fetcher.subscribedPartitions()[2];
-
-		// elements generate a watermark if the timestamp is a multiple of three
-
-		// elements for partition 1
-		fetcher.emitRecord(1L, part1, 1L);
-		fetcher.emitRecord(2L, part1, 2L);
-		fetcher.emitRecord(3L, part1, 3L);
-		assertEquals(3L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(3L, sourceContext.getLatestElement().getTimestamp());
-
-		// elements for partition 2
-		fetcher.emitRecord(12L, part2, 1L);
-		assertEquals(12L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(12L, sourceContext.getLatestElement().getTimestamp());
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(errorRef), sourceContext.getCheckpointLock());
 
-		// elements for partition 3
-		fetcher.emitRecord(101L, part3, 1L);
-		fetcher.emitRecord(102L, part3, 2L);
-		assertEquals(102L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(102L, sourceContext.getLatestElement().getTimestamp());
-
-		// now, we should have a watermark (this blocks until the periodic thread emitted the watermark)
-		assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp());
-
-		// advance partition 3
-		fetcher.emitRecord(1003L, part3, 3L);
-		fetcher.emitRecord(1004L, part3, 4L);
-		fetcher.emitRecord(1005L, part3, 5L);
-		assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(1005L, sourceContext.getLatestElement().getTimestamp());
-
-		// advance partition 1 beyond partition 2 - this bumps the watermark
-		fetcher.emitRecord(30L, part1, 4L);
-		assertEquals(30L, sourceContext.getLatestElement().getValue().longValue());
-		assertEquals(30L, sourceContext.getLatestElement().getTimestamp());
-		
-		// this blocks until the periodic thread emitted the watermark
-		assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp());
-
-		// advance partition 2 again - this bumps the watermark
-		fetcher.emitRecord(13L, part2, 2L);
-		fetcher.emitRecord(14L, part2, 3L);
-		fetcher.emitRecord(15L, part2, 3L);
-
-		// this blocks until the periodic thread emitted the watermark
-		long watermarkTs = sourceContext.getLatestWatermark().getTimestamp();
-		assertTrue(watermarkTs >= 13L && watermarkTs <= 15L);
+		try {
+			TestFetcher<Long> fetcher = new TestFetcher<>(
+					sourceContext, originalPartitions,
+					new SerializedValue<AssignerWithPeriodicWatermarks<Long>>(new PeriodicTestExtractor()),
+					null, new MockRuntimeContext(17, 3, config, timerService));
+	
+			final KafkaTopicPartitionState<Object> part1 = fetcher.subscribedPartitions()[0];
+			final KafkaTopicPartitionState<Object> part2 = fetcher.subscribedPartitions()[1];
+			final KafkaTopicPartitionState<Object> part3 = fetcher.subscribedPartitions()[2];
+	
+			// elements generate a watermark if the timestamp is a multiple of three
+	
+			// elements for partition 1
+			fetcher.emitRecord(1L, part1, 1L);
+			fetcher.emitRecord(2L, part1, 2L);
+			fetcher.emitRecord(3L, part1, 3L);
+			assertEquals(3L, sourceContext.getLatestElement().getValue().longValue());
+			assertEquals(3L, sourceContext.getLatestElement().getTimestamp());
+	
+			// elements for partition 2
+			fetcher.emitRecord(12L, part2, 1L);
+			assertEquals(12L, sourceContext.getLatestElement().getValue().longValue());
+			assertEquals(12L, sourceContext.getLatestElement().getTimestamp());
+	
+			// elements for partition 3
+			fetcher.emitRecord(101L, part3, 1L);
+			fetcher.emitRecord(102L, part3, 2L);
+			assertEquals(102L, sourceContext.getLatestElement().getValue().longValue());
+			assertEquals(102L, sourceContext.getLatestElement().getTimestamp());
+	
+			// now, we should have a watermark (this blocks until the periodic thread emitted the watermark)
+			assertEquals(3L, sourceContext.getLatestWatermark().getTimestamp());
+	
+			// advance partition 3
+			fetcher.emitRecord(1003L, part3, 3L);
+			fetcher.emitRecord(1004L, part3, 4L);
+			fetcher.emitRecord(1005L, part3, 5L);
+			assertEquals(1005L, sourceContext.getLatestElement().getValue().longValue());
+			assertEquals(1005L, sourceContext.getLatestElement().getTimestamp());
+	
+			// advance partition 1 beyond partition 2 - this bumps the watermark
+			fetcher.emitRecord(30L, part1, 4L);
+			assertEquals(30L, sourceContext.getLatestElement().getValue().longValue());
+			assertEquals(30L, sourceContext.getLatestElement().getTimestamp());
+			
+			// this blocks until the periodic thread emitted the watermark
+			assertEquals(12L, sourceContext.getLatestWatermark().getTimestamp());
+	
+			// advance partition 2 again - this bumps the watermark
+			fetcher.emitRecord(13L, part2, 2L);
+			fetcher.emitRecord(14L, part2, 3L);
+			fetcher.emitRecord(15L, part2, 3L);
+	
+			// this blocks until the periodic thread emitted the watermark
+			long watermarkTs = sourceContext.getLatestWatermark().getTimestamp();
+			assertTrue(watermarkTs >= 13L && watermarkTs <= 15L);
+		}
+		finally {
+			timerService.shutdownService();
+		}
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
index 5be4195..e1ec4cb 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
@@ -32,45 +32,46 @@ import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.metrics.MetricGroup;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.MockEnvironment;
 import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
-import org.apache.flink.util.Preconditions;
 
 import java.io.Serializable;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledFuture;
 
 @SuppressWarnings("deprecation")
 public class MockRuntimeContext extends StreamingRuntimeContext {
 
 	private final int numberOfParallelSubtasks;
 	private final int indexOfThisSubtask;
-	
-	private final ExecutionConfig execConfig;
 
-	private final TimeServiceProvider timerService;
+	private final ExecutionConfig execConfig;
 
+	private final TimeServiceProvider timeServiceProvider;
+	
 	public MockRuntimeContext(int numberOfParallelSubtasks, int indexOfThisSubtask) {
-		this(numberOfParallelSubtasks, indexOfThisSubtask, new ExecutionConfig(), new Object());
+		this(numberOfParallelSubtasks, indexOfThisSubtask, new ExecutionConfig());
 	}
 
 	public MockRuntimeContext(
-		int numberOfParallelSubtasks,
-		int indexOfThisSubtask,
-		ExecutionConfig execConfig,
-		Object checkpointLock) {
-
+			int numberOfParallelSubtasks,
+			int indexOfThisSubtask,
+			ExecutionConfig execConfig) {
+		this(numberOfParallelSubtasks, indexOfThisSubtask, execConfig, null);
+	}
+	
+	public MockRuntimeContext(
+			int numberOfParallelSubtasks,
+			int indexOfThisSubtask,
+			ExecutionConfig execConfig,
+			TimeServiceProvider timeServiceProvider) {
+		
 		super(new MockStreamOperator(),
 			new MockEnvironment("no", 4 * MemoryManager.DEFAULT_PAGE_SIZE, null, 16),
 			Collections.<String, Accumulator<?, ?>>emptyMap());
@@ -78,8 +79,7 @@ public class MockRuntimeContext extends StreamingRuntimeContext {
 		this.numberOfParallelSubtasks = numberOfParallelSubtasks;
 		this.indexOfThisSubtask = indexOfThisSubtask;
 		this.execConfig = execConfig;
-		this.timerService = DefaultTimeServiceProvider.
-			createForTesting(Executors.newSingleThreadScheduledExecutor(), checkpointLock);
+		this.timeServiceProvider = timeServiceProvider;
 	}
 
 	@Override
@@ -189,7 +189,11 @@ public class MockRuntimeContext extends StreamingRuntimeContext {
 
 	@Override
 	public TimeServiceProvider getTimeServiceProvider() {
-		return timerService;
+		if (timeServiceProvider == null) {
+			throw new UnsupportedOperationException();
+		} else {
+			return timeServiceProvider;
+		}
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
index abaf4e7..a290deb 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
@@ -35,9 +35,9 @@ public class StreamSourceContexts {
 	 * Depending on the {@link TimeCharacteristic}, this method will return the adequate
 	 * {@link org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}. That is:
 	 * <ul>
-	 * <li> {@link TimeCharacteristic#IngestionTime} = {@link AutomaticWatermarkContext}
-	 * <li> {@link TimeCharacteristic#ProcessingTime} = {@link NonTimestampContext}
-	 * <li> {@link TimeCharacteristic#EventTime} = {@link ManualWatermarkContext}
+	 *     <li>{@link TimeCharacteristic#IngestionTime} = {@link AutomaticWatermarkContext}</li>
+	 *     <li>{@link TimeCharacteristic#ProcessingTime} = {@link NonTimestampContext}</li>
+	 *     <li>{@link TimeCharacteristic#EventTime} = {@link ManualWatermarkContext}</li>
 	 * </ul>
 	 * */
 	public static <OUT> SourceFunction.SourceContext<OUT> getSourceContext(

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index 85e9297..2dbc6d4 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -83,7 +83,9 @@ public class StreamInputProcessor<IN> {
 	private Counter numRecordsIn;
 
 	@SuppressWarnings("unchecked")
-	public StreamInputProcessor(InputGate[] inputGates, TypeSerializer<IN> inputSerializer,
+	public StreamInputProcessor(
+			InputGate[] inputGates,
+			TypeSerializer<IN> inputSerializer,
 			StatefulTask checkpointedTask,
 			CheckpointingMode checkpointMode,
 			IOManager ioManager,

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
index 4c55055..a8125c3 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.streaming.runtime.tasks;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsynchronousException.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsynchronousException.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsynchronousException.java
index 311e0cd..cda0511 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsynchronousException.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsynchronousException.java
@@ -15,22 +15,27 @@
  * 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.annotation.Internal;
 
 /**
- * {@code RuntimeException} for wrapping exceptions that are thrown in Threads that are not the
- * main compute Thread.
+ * An exception for wrapping exceptions that are thrown by an operator in threads other than the
+ * main compute thread of that operator. 
  */
 @Internal
-public class AsynchronousException extends RuntimeException {
+public class AsynchronousException extends Exception {
 	private static final long serialVersionUID = 1L;
 
 	public AsynchronousException(Throwable cause) {
 		super(cause);
 	}
 
+	public AsynchronousException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
 	@Override
 	public String toString() {
 		return "AsynchronousException{" + getCause() + "}";

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
index 9534b3c..5664eac 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
@@ -17,15 +17,15 @@
 
 package org.apache.flink.streaming.runtime.tasks;
 
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.util.Preconditions;
 
-import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * A {@link TimeServiceProvider} which assigns as current processing time the result of calling
  * {@link System#currentTimeMillis()} and registers timers using a {@link ScheduledThreadPoolExecutor}.
@@ -35,24 +35,34 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 	/** The containing task that owns this time service provider. */
 	private final AsyncExceptionHandler task;
 
+	/** The lock that timers acquire upon triggering */
 	private final Object checkpointLock;
 
 	/** The executor service that schedules and calls the triggers of this task*/
-	private final ScheduledExecutorService timerService;
+	private final ScheduledThreadPoolExecutor timerService;
+
 
-	public static DefaultTimeServiceProvider create(
-			AsyncExceptionHandler exceptionHandler,
-			ScheduledExecutorService executor,
-			Object checkpointLock) {
-		return new DefaultTimeServiceProvider(exceptionHandler, executor, checkpointLock);
+	public DefaultTimeServiceProvider(AsyncExceptionHandler failureHandler, Object checkpointLock) {
+		this(failureHandler, checkpointLock, null);
 	}
 
-	private DefaultTimeServiceProvider(AsyncExceptionHandler task,
-									ScheduledExecutorService threadPoolExecutor,
-									Object checkpointLock) {
-		this.task = Preconditions.checkNotNull(task);
-		this.timerService = Preconditions.checkNotNull(threadPoolExecutor);
-		this.checkpointLock = Preconditions.checkNotNull(checkpointLock);
+	public DefaultTimeServiceProvider(
+			AsyncExceptionHandler task,
+			Object checkpointLock,
+			ThreadFactory threadFactory) {
+		
+		this.task = checkNotNull(task);
+		this.checkpointLock = checkNotNull(checkpointLock);
+
+		if (threadFactory == null) {
+			this.timerService = new ScheduledThreadPoolExecutor(1);
+		} else {
+			this.timerService = new ScheduledThreadPoolExecutor(1, threadFactory);
+		}
+
+		// allow trigger tasks to be removed if all timers for
+		// that timestamp are removed by user
+		this.timerService.setRemoveOnCancelPolicy(true);
 	}
 
 	@Override
@@ -76,6 +86,13 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 		timerService.shutdownNow();
 	}
 
+	// safety net to destroy the thread pool
+	@Override
+	protected void finalize() throws Throwable {
+		super.finalize();
+		timerService.shutdownNow();
+	}
+
 	/**
 	 * Internal task that is invoked by the timer service and triggers the target.
 	 */
@@ -105,14 +122,4 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 			}
 		}
 	}
-
-	@VisibleForTesting
-	public static DefaultTimeServiceProvider createForTesting(ScheduledExecutorService executor, Object checkpointLock) {
-		return new DefaultTimeServiceProvider(new AsyncExceptionHandler() {
-			@Override
-			public void handleAsyncException(String message, Throwable exception) {
-				exception.printStackTrace();
-			}
-		}, executor, checkpointLock);
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
index cf8853e..0a6534b 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
@@ -64,7 +64,7 @@ public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamO
 		final Object lock = getCheckpointLock();
 		
 		while (running && inputProcessor.processInput(operator, lock)) {
-
+			// all the work happens in the "processInput" method
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 33317fa..040ec66 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -67,7 +67,7 @@ import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
 
 /**
  * Base class for all streaming tasks. A task is the unit of local processing that is deployed
@@ -223,15 +223,10 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 
 			// if the clock is not already set, then assign a default TimeServiceProvider
 			if (timerService == null) {
+				ThreadFactory timerThreadFactory =
+					new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName());
 
-				ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1,
-					new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()));
-
-				// allow trigger tasks to be removed if all timers for
-				// that timestamp are removed by user
-				executor.setRemoveOnCancelPolicy(true);
-
-				timerService = DefaultTimeServiceProvider.create(this, executor, getCheckpointLock());
+				timerService = new DefaultTimeServiceProvider(this, getCheckpointLock(), timerThreadFactory);
 			}
 
 			operatorChain = new OperatorChain<>(this, getEnvironment().getAccumulatorRegistry().getReadWriteReporter());
@@ -305,10 +300,7 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 			// stop all timers and threads
 			if (timerService != null) {
 				try {
-					if (!timerService.isTerminated()) {
-						LOG.info("Timer service is shutting down.");
-						timerService.shutdownService();
-					}
+					timerService.shutdownService();
 				}
 				catch (Throwable t) {
 					// catch and log the exception to not replace the original exception

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
index a21a2e1..81faec9 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
@@ -86,7 +86,7 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 		return isTerminated;
 	}
 
-	public int getNoOfRegisteredTimers() {
+	public int getNumRegisteredTimers() {
 		int count = 0;
 		for (List<Triggerable> tasks: registeredTasks.values()) {
 			count += tasks.size();

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
index 0197c53..fb08959 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
@@ -88,7 +88,7 @@ public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputS
 		final Object lock = getCheckpointLock();
 		
 		while (running && inputProcessor.processInput(operator, lock)) {
-
+			// all the work happens in the "processInput" method
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
index 0351978..8d3e621 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
@@ -15,6 +15,7 @@
  * 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.typeinfo.BasicTypeInfo;
@@ -28,6 +29,7 @@ import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
 import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
 import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
+
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -37,13 +39,14 @@ import org.powermock.modules.junit4.PowerMockRunner;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 @RunWith(PowerMockRunner.class)
-@PrepareForTest(ResultPartitionWriter.class)
+@PrepareForTest({ResultPartitionWriter.class})
 @PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"})
 public class TimeProviderTest {
 
@@ -52,8 +55,10 @@ public class TimeProviderTest {
 		final OneShotLatch latch = new OneShotLatch();
 
 		final Object lock = new Object();
-		TimeServiceProvider timeServiceProvider = DefaultTimeServiceProvider
-			.createForTesting(Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+		
+		TimeServiceProvider timeServiceProvider = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		final List<Long> timestamps = new ArrayList<>();
 
@@ -114,6 +119,8 @@ public class TimeProviderTest {
 			lastTs = timestamp;
 			counter++;
 		}
+
+		assertNull(error.get());
 	}
 
 	@Test
@@ -124,14 +131,14 @@ public class TimeProviderTest {
 
 		final Object lock = new Object();
 
-		TimeServiceProvider timeServiceProvider = DefaultTimeServiceProvider
-			.create(new AsyncExceptionHandler() {
+		TimeServiceProvider timeServiceProvider = new DefaultTimeServiceProvider(
+			new AsyncExceptionHandler() {
 				@Override
 				public void handleAsyncException(String message, Throwable exception) {
 					exceptionWasThrown.compareAndSet(false, true);
 					latch.trigger();
 				}
-			}, Executors.newSingleThreadScheduledExecutor(), lock);
+			}, lock);
 
 		long now = System.currentTimeMillis();
 		timeServiceProvider.registerTimer(now, new Triggerable() {
@@ -182,7 +189,7 @@ public class TimeProviderTest {
 			}
 		});
 
-		Assert.assertEquals(provider.getNoOfRegisteredTimers(), 4);
+		Assert.assertEquals(provider.getNumRegisteredTimers(), 4);
 
 		provider.setCurrentTime(100);
 		long seen = 0;
@@ -233,14 +240,30 @@ public class TimeProviderTest {
 			}
 		});
 
-		assertEquals(2, tp.getNoOfRegisteredTimers());
+		assertEquals(2, tp.getNumRegisteredTimers());
 
 		tp.setCurrentTime(35);
-		assertEquals(1, tp.getNoOfRegisteredTimers());
+		assertEquals(1, tp.getNumRegisteredTimers());
 
 		tp.setCurrentTime(40);
-		assertEquals(0, tp.getNoOfRegisteredTimers());
+		assertEquals(0, tp.getNumRegisteredTimers());
 
 		tp.shutdownService();
 	}
+
+	// ------------------------------------------------------------------------
+
+	public static class ReferenceSettingExceptionHandler implements AsyncExceptionHandler {
+
+		private final AtomicReference<Throwable> errorReference;
+
+		public ReferenceSettingExceptionHandler(AtomicReference<Throwable> errorReference) {
+			this.errorReference = errorReference;
+		}
+
+		@Override
+		public void handleAsyncException(String message, Throwable exception) {
+			errorReference.compareAndSet(null, exception);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
index 30f38e3..4c6d391 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
@@ -39,6 +39,7 @@ 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.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
@@ -58,7 +59,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -182,14 +183,11 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 
 	@Test
 	public void testWindowTriggerTimeAlignment() throws Exception {
-		final Object lock = new Object();
-		TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
-
 		try {
 			@SuppressWarnings("unchecked")
 			final Output<StreamRecord<String>> mockOut = mock(Output.class);
-			StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+			final TimeServiceProvider timerService = new NoOpTimerService();
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, new Object());
 
 			AccumulatingProcessingTimeWindowOperator<String, String, String> op;
 
@@ -201,11 +199,6 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
-			timerService.shutdownService();
-			timerService = DefaultTimeServiceProvider.createForTesting(
-				Executors.newSingleThreadScheduledExecutor(), lock);
-			mockTask = createMockTaskWithTimer(timerService, lock);
-
 			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -214,11 +207,6 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
-			timerService.shutdownService();
-			timerService = DefaultTimeServiceProvider.createForTesting(
-				Executors.newSingleThreadScheduledExecutor(), lock);
-			mockTask = createMockTaskWithTimer(timerService, lock);
-
 			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -227,11 +215,6 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
-			timerService.shutdownService();
-			timerService = DefaultTimeServiceProvider.createForTesting(
-				Executors.newSingleThreadScheduledExecutor(), lock);
-			mockTask = createMockTaskWithTimer(timerService, lock);
-
 			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -244,16 +227,15 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			e.printStackTrace();
 			fail(e.getMessage());
 		}
-		finally {
-			timerService.shutdownService();
-		}
 	}
 
 	@Test
 	public void testTumblingWindow() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final int windowSize = 50;
@@ -285,6 +267,8 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
 
 			List<Integer> result = out.getElements();
@@ -294,6 +278,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			for (int i = 0; i < numElements; i++) {
 				assertEquals(i, result.get(i).intValue());
 			}
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -307,8 +295,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testSlidingWindow() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
@@ -335,6 +325,8 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
 
 			// get and verify the result
@@ -361,6 +353,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 					lastCount = 1;
 				}
 			}
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		} finally {
 			timerService.shutdownService();
 		}
@@ -369,8 +365,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testTumblingWindowSingleElements() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
@@ -412,7 +410,13 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -426,8 +430,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testSlidingWindowSingleElements() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+		
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+			new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
@@ -460,7 +466,13 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -798,4 +810,12 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 		}
 		return result;
 	}
+
+	private static void shutdownTimerServiceAndWait(TimeServiceProvider timers) throws Exception {
+		timers.shutdownService();
+
+		while (!timers.isTerminated()) {
+			Thread.sleep(2);
+		}
+	} 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
index 7539c2d..88e28bc 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
@@ -40,14 +40,15 @@ import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
-
 import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
 import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+
 import org.junit.After;
 import org.junit.Test;
 
@@ -60,7 +61,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicReference;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -190,15 +191,12 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 
 	@Test
 	public void testWindowTriggerTimeAlignment() throws Exception {
-		final Object lock = new Object();
-		TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
-
 		try {
 			@SuppressWarnings("unchecked")
 			final Output<StreamRecord<String>> mockOut = mock(Output.class);
-			StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
-			
+			final TimeServiceProvider timerService = new NoOpTimerService();
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, new Object());
+
 			AggregatingProcessingTimeWindowOperator<String, String> op;
 
 			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
@@ -209,11 +207,6 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
-			timerService.shutdownService();
-			timerService = DefaultTimeServiceProvider.createForTesting(
-				Executors.newSingleThreadScheduledExecutor(), lock);
-			mockTask = createMockTaskWithTimer(timerService, lock);
-
 			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -222,11 +215,6 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
-			timerService.shutdownService();
-			timerService = DefaultTimeServiceProvider.createForTesting(
-				Executors.newSingleThreadScheduledExecutor(), lock);
-			mockTask = createMockTaskWithTimer(timerService, lock);
-
 			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -235,11 +223,6 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
-			timerService.shutdownService();
-			timerService = DefaultTimeServiceProvider.createForTesting(
-				Executors.newSingleThreadScheduledExecutor(), lock);
-			mockTask = createMockTaskWithTimer(timerService, lock);
-
 			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -251,16 +234,16 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
-		} finally {
-			timerService.shutdownService();
 		}
 	}
 
 	@Test
 	public void testTumblingWindowUniqueElements() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final int windowSize = 50;
@@ -297,6 +280,8 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
 
 
@@ -305,6 +290,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 				assertEquals(i, result.get(i).f0.intValue());
 				assertEquals(i, result.get(i).f1.intValue());
 			}
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -318,8 +307,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testTumblingWindowDuplicateElements() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 		try {
 			final int windowSize = 50;
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>(windowSize);
@@ -364,6 +355,8 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
 
 			// we have ideally one element per window. we may have more, when we emitted a value into the
@@ -373,6 +366,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			// deduplicate for more accurate checks
 			HashSet<Tuple2<Integer, Integer>> set = new HashSet<>(result);
 			assertTrue(set.size() == 10);
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -386,8 +383,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testSlidingWindow() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>(50);
@@ -418,6 +417,8 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
 
 			// get and verify the result
@@ -445,6 +446,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 					lastCount = 1;
 				}
 			}
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -458,8 +463,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testSlidingWindowSingleElements() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>(50);
@@ -504,7 +511,13 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			synchronized (lock) {
 				op.close();
 			}
+
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -518,8 +531,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testPropagateExceptionsFromProcessElement() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
-			Executors.newSingleThreadScheduledExecutor(), lock);
+		final AtomicReference<Throwable> error = new AtomicReference<>();
+
+		final TimeServiceProvider timerService = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(error), lock);
 
 		try {
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>();
@@ -556,7 +571,12 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 				assertTrue(e.getMessage().contains("Artificial Test Exception"));
 			}
 
+			shutdownTimerServiceAndWait(timerService);
 			op.dispose();
+
+			if (error.get() != null) {
+				throw new Exception(error.get());
+			}
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -971,8 +991,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 	}
 
 	private static StreamConfig createTaskConfig(KeySelector<?, ?> partitioner, TypeSerializer<?> keySerializer, int numberOfKeGroups) {
-		StreamConfig cfg = new StreamConfig(new Configuration());
-		return cfg;
+		return new StreamConfig(new Configuration());
 	}
 
 	@SuppressWarnings({"unchecked", "rawtypes"})
@@ -985,4 +1004,12 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 		}
 		return result;
 	}
+
+	private static void shutdownTimerServiceAndWait(TimeServiceProvider timers) throws Exception {
+		timers.shutdownService();
+
+		while (!timers.isTerminated()) {
+			Thread.sleep(2);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java
new file mode 100644
index 0000000..16e658e
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.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.runtime.operators.windowing;
+
+import org.apache.flink.streaming.runtime.operators.Triggerable;
+import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
+
+import java.util.concurrent.ScheduledFuture;
+
+class NoOpTimerService extends TimeServiceProvider {
+
+	private volatile boolean terminated;
+	
+	@Override
+	public long getCurrentProcessingTime() {
+		return System.currentTimeMillis();
+	}
+
+	@Override
+	public ScheduledFuture<?> registerTimer(long timestamp, Triggerable target) {
+		return null;
+	}
+
+	@Override
+	public boolean isTerminated() {
+		return terminated;
+	}
+
+	@Override
+	public void shutdownService() throws Exception {
+		terminated = true;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index d8a0ee2..9d8e6a5 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -42,12 +42,12 @@ import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
 import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
+
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import java.util.Collection;
 import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.Executors;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
@@ -145,7 +145,7 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
 				@Override
 				public CheckpointStreamFactory answer(InvocationOnMock invocationOnMock) throws Throwable {
 
-					final StreamOperator operator = (StreamOperator) invocationOnMock.getArguments()[0];
+					final StreamOperator<?> operator = (StreamOperator<?>) invocationOnMock.getArguments()[0];
 					return stateBackend.createStreamFactory(new JobID(), operator.getClass().getSimpleName());
 				}
 			}).when(mockTask).createCheckpointStreamFactory(any(StreamOperator.class));
@@ -154,7 +154,7 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
 		}
 
 		timeServiceProvider = testTimeProvider != null ? testTimeProvider :
-			DefaultTimeServiceProvider.create(mockTask, Executors.newSingleThreadScheduledExecutor(), this.checkpointLock);
+			new DefaultTimeServiceProvider(mockTask, this.checkpointLock);
 
 		doAnswer(new Answer<TimeServiceProvider>() {
 			@Override


[15/17] flink git commit: [FLINK-4749] [streaming api] Remove redundant processing time timer sets from window operator

Posted by se...@apache.org.
[FLINK-4749] [streaming api] Remove redundant processing time timer sets from window operator


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

Branch: refs/heads/master
Commit: 47e4977442f7045baf29ce7dc772b7f7aff65343
Parents: 9d24d51
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Oct 4 23:48:31 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 20:04:34 2016 +0200

----------------------------------------------------------------------
 .../windowing/EvictingWindowOperator.java       |  79 +++++------
 .../operators/windowing/WindowOperator.java     | 140 +++++++++----------
 .../runtime/tasks/TestTimeServiceProvider.java  |  72 +++++++++-
 .../operators/windowing/WindowOperatorTest.java |  24 ++--
 4 files changed, 175 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/47e49774/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
index a838faa..6609e4d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
@@ -262,60 +262,53 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
 
 	@Override
 	public void trigger(long time) throws Exception {
-		boolean fire;
+		Timer<K, W> timer;
 
-		//Remove information about the triggering task
-		processingTimeTimerFutures.remove(time);
-		processingTimeTimerTimestamps.remove(time, processingTimeTimerTimestamps.count(time));
+		while ((timer = processingTimeTimersQueue.peek()) != null && timer.timestamp <= time) {
 
-		do {
-			Timer<K, W> timer = processingTimeTimersQueue.peek();
-			if (timer != null && timer.timestamp <= time) {
-				fire = true;
+			processingTimeTimers.remove(timer);
+			processingTimeTimersQueue.remove();
 
-				processingTimeTimers.remove(timer);
-				processingTimeTimersQueue.remove();
+			context.key = timer.key;
+			context.window = timer.window;
+			setKeyContext(timer.key);
 
-				context.key = timer.key;
-				context.window = timer.window;
-				setKeyContext(timer.key);
-
-				ListState<StreamRecord<IN>> windowState;
-				MergingWindowSet<W> mergingWindows = null;
-
-				if (windowAssigner instanceof MergingWindowAssigner) {
-					mergingWindows = getMergingWindowSet();
-					W stateWindow = mergingWindows.getStateWindow(context.window);
-					if (stateWindow == null) {
-						// then the window is already purged and this is a cleanup
-						// timer set due to allowed lateness that has nothing to clean,
-						// so it is safe to just ignore
-						continue;
-					}
-					windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
-				} else {
-					windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
-				}
+			ListState<StreamRecord<IN>> windowState;
+			MergingWindowSet<W> mergingWindows = null;
 
-				Iterable<StreamRecord<IN>> contents = windowState.get();
-				if (contents == null) {
-					// if we have no state, there is nothing to do
+			if (windowAssigner instanceof MergingWindowAssigner) {
+				mergingWindows = getMergingWindowSet();
+				W stateWindow = mergingWindows.getStateWindow(context.window);
+				if (stateWindow == null) {
+					// then the window is already purged and this is a cleanup
+					// timer set due to allowed lateness that has nothing to clean,
+					// so it is safe to just ignore
 					continue;
 				}
+				windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
+			} else {
+				windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
+			}
 
-				TriggerResult triggerResult = context.onProcessingTime(timer.timestamp);
-				if (triggerResult.isFire()) {
-					fire(context.window, contents);
-				}
+			Iterable<StreamRecord<IN>> contents = windowState.get();
+			if (contents == null) {
+				// if we have no state, there is nothing to do
+				continue;
+			}
 
-				if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
-					cleanup(context.window, windowState, mergingWindows);
-				}
+			TriggerResult triggerResult = context.onProcessingTime(timer.timestamp);
+			if (triggerResult.isFire()) {
+				fire(context.window, contents);
+			}
 
-			} else {
-				fire = false;
+			if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
+				cleanup(context.window, windowState, mergingWindows);
 			}
-		} while (fire);
+		}
+
+		if (timer != null) {
+			nextTimer = getTimerService().registerTimer(timer.timestamp, this);
+		}
 	}
 
 	private void fire(W window, Iterable<StreamRecord<IN>> contents) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/47e49774/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index f010822..de316e7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.streaming.runtime.operators.windowing;
 
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.Multiset;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
@@ -151,7 +149,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 */
 	protected transient TimestampedCollector<OUT> timestampedCollector;
 
-	protected transient Map<Long, ScheduledFuture<?>> processingTimeTimerFutures;
+	protected transient ScheduledFuture<?> nextTimer;
 
 	/**
 	 * To keep track of the current watermark so that we can immediately fire if a trigger
@@ -172,7 +170,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 */
 	protected transient PriorityQueue<Timer<K, W>> processingTimeTimersQueue;
 	protected transient Set<Timer<K, W>> processingTimeTimers;
-	protected transient Multiset<Long> processingTimeTimerTimestamps;
 
 	/**
 	 * Current waiting watermark callbacks.
@@ -219,7 +216,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		inputSerializer = (TypeSerializer<IN>) type.createSerializer(executionConfig);
 	}
 
-	@SuppressWarnings("unchecked")
 	@Override
 	public final void open() throws Exception {
 		super.open();
@@ -237,13 +233,9 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		}
 		if (processingTimeTimers == null) {
 			processingTimeTimers = new HashSet<>();
-			processingTimeTimerTimestamps = HashMultiset.create();
 			processingTimeTimersQueue = new PriorityQueue<>(100);
 		}
 
-		//ScheduledFutures are not checkpointed
-		processingTimeTimerFutures = new HashMap<>();
-
 		context = new Context(null, null);
 
 		windowAssignerContext = new WindowAssigner.WindowAssignerContext() {
@@ -261,6 +253,12 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	@Override
 	public final void close() throws Exception {
 		super.close();
+
+		if (nextTimer != null) {
+			nextTimer.cancel(false);
+			nextTimer = null;
+		}
+
 		timestampedCollector = null;
 		watermarkTimers = null;
 		watermarkTimersQueue = null;
@@ -274,6 +272,12 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	@Override
 	public void dispose() throws Exception {
 		super.dispose();
+
+		if (nextTimer != null) {
+			nextTimer.cancel(false);
+			nextTimer = null;
+		}
+
 		timestampedCollector = null;
 		watermarkTimers = null;
 		watermarkTimersQueue = null;
@@ -459,60 +463,53 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 
 	@Override
 	public void trigger(long time) throws Exception {
-		boolean fire;
-
-		//Remove information about the triggering task
-		processingTimeTimerFutures.remove(time);
-		processingTimeTimerTimestamps.remove(time, processingTimeTimerTimestamps.count(time));
+		Timer<K, W> timer;
 
-		do {
-			Timer<K, W> timer = processingTimeTimersQueue.peek();
-			if (timer != null && timer.timestamp <= time) {
-				fire = true;
+		while ((timer = processingTimeTimersQueue.peek()) != null && timer.timestamp <= time) {
 
-				processingTimeTimers.remove(timer);
-				processingTimeTimersQueue.remove();
+			processingTimeTimers.remove(timer);
+			processingTimeTimersQueue.remove();
 
-				context.key = timer.key;
-				context.window = timer.window;
-				setKeyContext(timer.key);
+			context.key = timer.key;
+			context.window = timer.window;
+			setKeyContext(timer.key);
 
-				AppendingState<IN, ACC> windowState;
-				MergingWindowSet<W> mergingWindows = null;
-
-				if (windowAssigner instanceof MergingWindowAssigner) {
-					mergingWindows = getMergingWindowSet();
-					W stateWindow = mergingWindows.getStateWindow(context.window);
-					if (stateWindow == null) {
-						// then the window is already purged and this is a cleanup
-						// timer set due to allowed lateness that has nothing to clean,
-						// so it is safe to just ignore
-						continue;
-					}
-					windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
-				} else {
-					windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
-				}
+			AppendingState<IN, ACC> windowState;
+			MergingWindowSet<W> mergingWindows = null;
 
-				ACC contents = windowState.get();
-				if (contents == null) {
-					// if we have no state, there is nothing to do
+			if (windowAssigner instanceof MergingWindowAssigner) {
+				mergingWindows = getMergingWindowSet();
+				W stateWindow = mergingWindows.getStateWindow(context.window);
+				if (stateWindow == null) {
+					// then the window is already purged and this is a cleanup
+					// timer set due to allowed lateness that has nothing to clean,
+					// so it is safe to just ignore
 					continue;
 				}
+				windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor);
+			} else {
+				windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor);
+			}
 
-				TriggerResult triggerResult = context.onProcessingTime(timer.timestamp);
-				if (triggerResult.isFire()) {
-					fire(context.window, contents);
-				}
+			ACC contents = windowState.get();
+			if (contents == null) {
+				// if we have no state, there is nothing to do
+				continue;
+			}
 
-				if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
-					cleanup(context.window, windowState, mergingWindows);
-				}
+			TriggerResult triggerResult = context.onProcessingTime(timer.timestamp);
+			if (triggerResult.isFire()) {
+				fire(context.window, contents);
+			}
 
-			} else {
-				fire = false;
+			if (triggerResult.isPurge() || (!windowAssigner.isEventTime() && isCleanupTime(context.window, timer.timestamp))) {
+				cleanup(context.window, windowState, mergingWindows);
 			}
-		} while (fire);
+		}
+
+		if (timer != null) {
+			nextTimer = getTimerService().registerTimer(timer.timestamp, this);
+		}
 	}
 
 	/**
@@ -719,14 +716,21 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		@Override
 		public void registerProcessingTimeTimer(long time) {
 			Timer<K, W> timer = new Timer<>(time, key, window);
+
 			// make sure we only put one timer per key into the queue
 			if (processingTimeTimers.add(timer)) {
+
+				Timer<K, W> oldHead = processingTimeTimersQueue.peek();
+				long nextTriggerTime = oldHead != null ? oldHead.timestamp : Long.MAX_VALUE; 
+
 				processingTimeTimersQueue.add(timer);
-				//If this is the first timer added for this timestamp register a TriggerTask
-				if (processingTimeTimerTimestamps.add(time, 1) == 0) {
-					ScheduledFuture<?> scheduledFuture = WindowOperator.this.getTimerService()
-						.registerTimer(time, WindowOperator.this);
-					processingTimeTimerFutures.put(time, scheduledFuture);
+
+				// check if we need to re-schedule our timer to earlier
+				if (time < nextTriggerTime) {
+					if (nextTimer != null) {
+						nextTimer.cancel(false);
+					}
+					nextTimer = getTimerService().registerTimer(time, WindowOperator.this);
 				}
 			}
 		}
@@ -746,14 +750,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			if (processingTimeTimers.remove(timer)) {
 				processingTimeTimersQueue.remove(timer);
 			}
-
-			//If there are no timers left for this timestamp, remove it from queue and cancel TriggerTask
-			if (processingTimeTimerTimestamps.remove(time,1) == 1) {
-				ScheduledFuture<?> triggerTaskFuture = processingTimeTimerFutures.remove(timer.timestamp);
-				if (triggerTaskFuture != null && !triggerTaskFuture.isDone()) {
-					triggerTaskFuture.cancel(false);
-				}
-			}
 		}
 
 		@Override
@@ -904,12 +900,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			processingTimeTimers.add(timer);
 		}
 
-		int numProcessingTimeTimerTimestamp = in.readInt();
-		processingTimeTimerTimestamps = HashMultiset.create();
-		for (int i = 0; i< numProcessingTimeTimerTimestamp; i++) {
-			long timestamp = in.readLong();
-			int count = in.readInt();
-			processingTimeTimerTimestamps.add(timestamp, count);
+		if (numProcessingTimeTimers > 0) {
+			nextTimer = getTimerService().registerTimer(processingTimeTimersQueue.peek().timestamp, this);
 		}
 	}
 
@@ -927,12 +919,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 			windowSerializer.serialize(timer.window, out);
 			out.writeLong(timer.timestamp);
 		}
-
-		out.writeInt(processingTimeTimerTimestamps.entrySet().size());
-		for (Multiset.Entry<Long> timerTimestampCounts: processingTimeTimerTimestamps.entrySet()) {
-			out.writeLong(timerTimestampCounts.getElement());
-			out.writeInt(timerTimestampCounts.getCount());
-		}
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/47e49774/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
index 81faec9..f4bead9 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
@@ -24,7 +24,11 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 /**
  * This is a {@link TimeServiceProvider} used <b>strictly for testing</b> the
@@ -32,13 +36,14 @@ import java.util.concurrent.ScheduledFuture;
  * */
 public class TestTimeServiceProvider extends TimeServiceProvider {
 
-	private long currentTime = 0;
+	private volatile long currentTime = 0;
 
-	private boolean isTerminated = false;
+	private volatile boolean isTerminated;
 
 	// sorts the timers by timestamp so that they are processed in the correct order.
-	private Map<Long, List<Triggerable>> registeredTasks = new TreeMap<>();
+	private final Map<Long, List<Triggerable>> registeredTasks = new TreeMap<>();
 
+	
 	public void setCurrentTime(long timestamp) throws Exception {
 		this.currentTime = timestamp;
 
@@ -72,13 +77,26 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 
 	@Override
 	public ScheduledFuture<?> registerTimer(long timestamp, Triggerable target) {
+		if (isTerminated) {
+			throw new IllegalStateException("terminated");
+		}
+
+		if (timestamp <= currentTime) {
+			try {
+				target.trigger(timestamp);
+			} catch (Exception e) {
+				throw new RuntimeException(e);
+			}
+		}
+
 		List<Triggerable> tasks = registeredTasks.get(timestamp);
 		if (tasks == null) {
 			tasks = new ArrayList<>();
 			registeredTasks.put(timestamp, tasks);
 		}
 		tasks.add(target);
-		return null;
+
+		return new DummyFuture();
 	}
 
 	@Override
@@ -86,6 +104,11 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 		return isTerminated;
 	}
 
+	@Override
+	public void shutdownService() throws Exception {
+		isTerminated = true;
+	}
+
 	public int getNumRegisteredTimers() {
 		int count = 0;
 		for (List<Triggerable> tasks: registeredTasks.values()) {
@@ -94,8 +117,43 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 		return count;
 	}
 
-	@Override
-	public void shutdownService() throws Exception {
-		this.isTerminated = true;
+	// ------------------------------------------------------------------------
+
+	private static class DummyFuture implements ScheduledFuture<Object> {
+
+		@Override
+		public long getDelay(TimeUnit unit) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public int compareTo(Delayed o) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public boolean cancel(boolean mayInterruptIfRunning) {
+			return true;
+		}
+
+		@Override
+		public boolean isCancelled() {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public boolean isDone() {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public Object get() throws InterruptedException, ExecutionException {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public Object get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+			throw new UnsupportedOperationException();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/47e49774/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index fd73bcc..e98bc91 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -876,6 +876,9 @@ public class WindowOperatorTest extends TestLogger {
 		final int WINDOW_SLIDE = 1;
 
 		TypeInformation<Tuple2<String, Integer>> inputType = TypeInfoParser.parse("Tuple2<String, Integer>");
+		TestTimeServiceProvider timer = new TestTimeServiceProvider();
+
+		TestTimeServiceProvider timer = new TestTimeServiceProvider();
 
 		ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents",
 				new SumReducer(),
@@ -893,9 +896,10 @@ public class WindowOperatorTest extends TestLogger {
 
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> testHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(operator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO); ;
-
-		operator.setInputType(inputType, new ExecutionConfig());
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						operator, new ExecutionConfig(), timer,
+						new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
+		
 		testHarness.open();
 
 		WindowOperator.Timer<String, TimeWindow> timer1 = new WindowOperator.Timer<>(1L, "key1", new TimeWindow(1L, 2L));
@@ -907,12 +911,7 @@ public class WindowOperatorTest extends TestLogger {
 		operator.processingTimeTimersQueue.add(timer1);
 		operator.processingTimeTimersQueue.add(timer2);
 		operator.processingTimeTimersQueue.add(timer3);
-
-		operator.processingTimeTimerTimestamps.add(1L, 10);
-		operator.processingTimeTimerTimestamps.add(2L, 5);
-		operator.processingTimeTimerTimestamps.add(3L, 1);
-
-
+		
 		StreamStateHandle snapshot = testHarness.snapshot(0, 0);
 
 		WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> otherOperator = new WindowOperator<>(
@@ -926,9 +925,9 @@ public class WindowOperatorTest extends TestLogger {
 				0);
 
 		OneInputStreamOperatorTestHarness<Tuple2<String, Integer>, Tuple2<String, Integer>> otherTestHarness =
-				new KeyedOneInputStreamOperatorTestHarness<>(otherOperator, new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
-
-		otherOperator.setInputType(inputType, new ExecutionConfig());
+				new KeyedOneInputStreamOperatorTestHarness<>(
+						otherOperator, new ExecutionConfig(), timer,
+						new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO);
 
 		otherTestHarness.setup();
 		otherTestHarness.restore(snapshot);
@@ -936,7 +935,6 @@ public class WindowOperatorTest extends TestLogger {
 
 		Assert.assertEquals(operator.processingTimeTimers, otherOperator.processingTimeTimers);
 		Assert.assertArrayEquals(operator.processingTimeTimersQueue.toArray(), otherOperator.processingTimeTimersQueue.toArray());
-		Assert.assertEquals(operator.processingTimeTimerTimestamps, otherOperator.processingTimeTimerTimestamps);
 	}
 
 	@Test


[09/17] flink git commit: [FLINK-4748] [streaming api] Make timers in Ingestion Time source context properly cancelable.

Posted by se...@apache.org.
[FLINK-4748] [streaming api] Make timers in Ingestion Time source context properly cancelable.


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

Branch: refs/heads/master
Commit: dd3416fde7b934584d4e18bc54d79ed7858556c2
Parents: beb31fc
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Oct 4 22:29:28 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:14 2016 +0200

----------------------------------------------------------------------
 .../api/operators/StreamSourceContexts.java     | 31 ++++++++++++--------
 1 file changed, 19 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/dd3416fd/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
index a290deb..d0c4e15 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
@@ -14,6 +14,7 @@
  * 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.TimeCharacteristic;
@@ -115,9 +116,9 @@ public class StreamSourceContexts {
 		private final Output<StreamRecord<T>> output;
 		private final StreamRecord<T> reuse;
 
-		private final ScheduledFuture<?> watermarkTimer;
 		private final long watermarkInterval;
 
+		private volatile ScheduledFuture<?> nextWatermarkTimer;
 		private volatile long nextWatermarkTime;
 
 		private AutomaticWatermarkContext(
@@ -130,13 +131,13 @@ public class StreamSourceContexts {
 			this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpoint lock cannot be null.");
 			this.output = Preconditions.checkNotNull(output, "The output cannot be null.");
 
-			Preconditions.checkArgument(watermarkInterval > 1L, "The watermark interval cannot be smaller than 1 ms.");
+			Preconditions.checkArgument(watermarkInterval >= 1L, "The watermark interval cannot be smaller than 1 ms.");
 			this.watermarkInterval = watermarkInterval;
 
 			this.reuse = new StreamRecord<>(null);
 
 			long now = this.timeService.getCurrentProcessingTime();
-			this.watermarkTimer = this.timeService.registerTimer(now + watermarkInterval,
+			this.nextWatermarkTimer = this.timeService.registerTimer(now + watermarkInterval,
 				new WatermarkEmittingTask(this.timeService, lock, output));
 		}
 
@@ -178,8 +179,9 @@ public class StreamSourceContexts {
 				}
 
 				// we can shutdown the timer now, no watermarks will be needed any more
-				if (watermarkTimer != null) {
-					watermarkTimer.cancel(true);
+				final ScheduledFuture<?> nextWatermarkTimer = this.nextWatermarkTimer;
+				if (nextWatermarkTimer != null) {
+					nextWatermarkTimer.cancel(true);
 				}
 			}
 		}
@@ -191,8 +193,9 @@ public class StreamSourceContexts {
 
 		@Override
 		public void close() {
-			if (watermarkTimer != null) {
-				watermarkTimer.cancel(true);
+			final ScheduledFuture<?> nextWatermarkTimer = this.nextWatermarkTimer;
+			if (nextWatermarkTimer != null) {
+				nextWatermarkTimer.cancel(true);
 			}
 		}
 
@@ -202,10 +205,13 @@ public class StreamSourceContexts {
 			private final Object lock;
 			private final Output<StreamRecord<T>> output;
 
-			private WatermarkEmittingTask(TimeServiceProvider timeService, Object checkpointLock, Output<StreamRecord<T>> output) {
-				this.timeService = Preconditions.checkNotNull(timeService, "Time Service cannot be null.");
-				this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpoint lock cannot be null.");
-				this.output = Preconditions.checkNotNull(output, "The output cannot be null.");
+			private WatermarkEmittingTask(
+					TimeServiceProvider timeService,
+					Object checkpointLock,
+					Output<StreamRecord<T>> output) {
+				this.timeService = timeService;
+				this.lock = checkpointLock;
+				this.output = output;
 			}
 
 			@Override
@@ -227,7 +233,8 @@ public class StreamSourceContexts {
 				}
 
 				long nextWatermark = currentTime + watermarkInterval;
-				this.timeService.registerTimer(nextWatermark, new WatermarkEmittingTask(this.timeService, lock, output));
+				nextWatermarkTimer = this.timeService.registerTimer(
+						nextWatermark, new WatermarkEmittingTask(this.timeService, lock, output));
 			}
 		}
 	}


[13/17] flink git commit: [FLINK-4750] [runtime] Cleanly await end of all currently executing processing time timers when finite streams finish.

Posted by se...@apache.org.
[FLINK-4750] [runtime] Cleanly await end of all currently executing processing time timers when finite streams finish.


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

Branch: refs/heads/master
Commit: 8aea8c8f427f5511c6064abbc4b85a3ef106743a
Parents: 1cd8d4f
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Oct 5 14:33:01 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 20:04:34 2016 +0200

----------------------------------------------------------------------
 .../tasks/DefaultTimeServiceProvider.java       | 151 +++++++++++++++-
 .../streaming/runtime/tasks/StreamTask.java     |   3 +
 .../runtime/tasks/TestTimeServiceProvider.java  |  55 +++---
 .../runtime/tasks/TimeServiceProvider.java      |  60 +++++--
 .../operators/windowing/NoOpTimerService.java   |   7 +-
 .../tasks/DefaultTimeServiceProviderTest.java   | 179 +++++++++++++++++++
 6 files changed, 414 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8aea8c8f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
index 5664eac..d2c743f 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
@@ -17,12 +17,20 @@
 
 package org.apache.flink.streaming.runtime.tasks;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 
+import javax.annotation.Nonnull;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -32,6 +40,12 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class DefaultTimeServiceProvider extends TimeServiceProvider {
 
+	private static final int STATUS_ALIVE = 0;
+	private static final int STATUS_QUIESCED = 1;
+	private static final int STATUS_SHUTDOWN = 2;
+
+	// ------------------------------------------------------------------------
+
 	/** The containing task that owns this time service provider. */
 	private final AsyncExceptionHandler task;
 
@@ -41,6 +55,8 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 	/** The executor service that schedules and calls the triggers of this task*/
 	private final ScheduledThreadPoolExecutor timerService;
 
+	private final AtomicInteger status;
+
 
 	public DefaultTimeServiceProvider(AsyncExceptionHandler failureHandler, Object checkpointLock) {
 		this(failureHandler, checkpointLock, null);
@@ -50,19 +66,24 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 			AsyncExceptionHandler task,
 			Object checkpointLock,
 			ThreadFactory threadFactory) {
-		
+
 		this.task = checkNotNull(task);
 		this.checkpointLock = checkNotNull(checkpointLock);
 
+		this.status = new AtomicInteger(STATUS_ALIVE);
+
 		if (threadFactory == null) {
 			this.timerService = new ScheduledThreadPoolExecutor(1);
 		} else {
 			this.timerService = new ScheduledThreadPoolExecutor(1, threadFactory);
 		}
 
-		// allow trigger tasks to be removed if all timers for
-		// that timestamp are removed by user
+		// tasks should be removed if the future is canceled
 		this.timerService.setRemoveOnCancelPolicy(true);
+
+		// make sure shutdown removes all pending tasks
+		this.timerService.setContinueExistingPeriodicTasksAfterShutdownPolicy(false);
+		this.timerService.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
 	}
 
 	@Override
@@ -73,17 +94,50 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 	@Override
 	public ScheduledFuture<?> registerTimer(long timestamp, Triggerable target) {
 		long delay = Math.max(timestamp - getCurrentProcessingTime(), 0);
-		return timerService.schedule(new TriggerTask(task, checkpointLock, target, timestamp), delay, TimeUnit.MILLISECONDS);
+
+		// we directly try to register the timer and only react to the status on exception
+		// that way we save unnecessary volatile accesses for each timer
+		try {
+			return timerService.schedule(
+					new TriggerTask(task, checkpointLock, target, timestamp), delay, TimeUnit.MILLISECONDS);
+		}
+		catch (RejectedExecutionException e) {
+			final int status = this.status.get();
+			if (status == STATUS_QUIESCED) {
+				return new NeverCompleteFuture(delay);
+			}
+			else if (status == STATUS_SHUTDOWN) {
+				throw new IllegalStateException("Timer service is shut down");
+			}
+			else {
+				// something else happened, so propagate the exception
+				throw e;
+			}
+		}
 	}
 
 	@Override
 	public boolean isTerminated() {
-		return timerService.isTerminated();
+		return status.get() == STATUS_SHUTDOWN;
 	}
 
 	@Override
-	public void shutdownService() throws Exception {
-		timerService.shutdownNow();
+	public void quiesceAndAwaitPending() throws InterruptedException {
+		if (status.compareAndSet(STATUS_ALIVE, STATUS_QUIESCED)) {
+			timerService.shutdown();
+
+			// await forever (almost)
+			timerService.awaitTermination(365, TimeUnit.DAYS);
+		}
+	}
+
+	@Override
+	public void shutdownService() {
+		if (status.compareAndSet(STATUS_ALIVE, STATUS_SHUTDOWN) || 
+				status.compareAndSet(STATUS_QUIESCED, STATUS_SHUTDOWN))
+		{
+			timerService.shutdownNow();
+		}
 	}
 
 	// safety net to destroy the thread pool
@@ -93,6 +147,18 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 		timerService.shutdownNow();
 	}
 
+	@VisibleForTesting
+	int getNumTasksScheduled() {
+		BlockingQueue<?> queue = timerService.getQueue();
+		if (queue == null) {
+			return 0;
+		} else {
+			return queue.size();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Internal task that is invoked by the timer service and triggers the target.
 	 */
@@ -122,4 +188,75 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 			}
 		}
 	}
+
+	// ------------------------------------------------------------------------
+
+	private static final class NeverCompleteFuture implements ScheduledFuture<Object> {
+
+		private final Object lock = new Object();
+
+		private final long delayMillis;
+
+		private volatile boolean canceled;
+
+
+		private NeverCompleteFuture(long delayMillis) {
+			this.delayMillis = delayMillis;
+		}
+
+		@Override
+		public long getDelay(@Nonnull TimeUnit unit) {
+			return unit.convert(delayMillis, TimeUnit.MILLISECONDS);
+		}
+
+		@Override
+		public int compareTo(@Nonnull Delayed o) {
+			long otherMillis = o.getDelay(TimeUnit.MILLISECONDS);
+			return Long.compare(this.delayMillis, otherMillis);
+		}
+
+		@Override
+		public boolean cancel(boolean mayInterruptIfRunning) {
+			synchronized (lock) {
+				canceled = true;
+				lock.notifyAll();
+			}
+			return true;
+		}
+
+		@Override
+		public boolean isCancelled() {
+			return canceled;
+		}
+
+		@Override
+		public boolean isDone() {
+			return false;
+		}
+
+		@Override
+		public Object get() throws InterruptedException {
+			synchronized (lock) {
+				while (!canceled) {
+					lock.wait();
+				}
+			}
+			throw new CancellationException();
+		}
+
+		@Override
+		public Object get(long timeout, @Nonnull TimeUnit unit) throws InterruptedException, TimeoutException {
+			synchronized (lock) {
+				while (!canceled) {
+					unit.timedWait(lock, timeout);
+				}
+
+				if (canceled) {
+					throw new CancellationException();
+				} else {
+					throw new TimeoutException();
+				}
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8aea8c8f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 040ec66..ff14249 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -269,6 +269,9 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 			isRunning = true;
 			run();
 
+			// make sure all timers finish and no new timers can come
+			timerService.quiesceAndAwaitPending();
+
 			LOG.debug("Finished task {}", getName());
 
 			// make sure no further checkpoint and notification actions happen.

http://git-wip-us.apache.org/repos/asf/flink/blob/8aea8c8f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
index f4bead9..9eb6cd1 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestTimeServiceProvider.java
@@ -39,6 +39,7 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 	private volatile long currentTime = 0;
 
 	private volatile boolean isTerminated;
+	private volatile boolean isQuiesced;
 
 	// sorts the timers by timestamp so that they are processed in the correct order.
 	private final Map<Long, List<Triggerable>> registeredTasks = new TreeMap<>();
@@ -47,25 +48,27 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 	public void setCurrentTime(long timestamp) throws Exception {
 		this.currentTime = timestamp;
 
-		// decide which timers to fire and put them in a list
-		// we do not fire them here to be able to accommodate timers
-		// that register other timers.
-
-		Iterator<Map.Entry<Long, List<Triggerable>>> it = registeredTasks.entrySet().iterator();
-		List<Map.Entry<Long, List<Triggerable>>> toRun = new ArrayList<>();
-		while (it.hasNext()) {
-			Map.Entry<Long, List<Triggerable>> t = it.next();
-			if (t.getKey() <= this.currentTime) {
-				toRun.add(t);
-				it.remove();
+		if (!isQuiesced) {
+			// decide which timers to fire and put them in a list
+			// we do not fire them here to be able to accommodate timers
+			// that register other timers.
+	
+			Iterator<Map.Entry<Long, List<Triggerable>>> it = registeredTasks.entrySet().iterator();
+			List<Map.Entry<Long, List<Triggerable>>> toRun = new ArrayList<>();
+			while (it.hasNext()) {
+				Map.Entry<Long, List<Triggerable>> t = it.next();
+				if (t.getKey() <= this.currentTime) {
+					toRun.add(t);
+					it.remove();
+				}
 			}
-		}
-
-		// now do the actual firing.
-		for (Map.Entry<Long, List<Triggerable>> tasks: toRun) {
-			long now = tasks.getKey();
-			for (Triggerable task: tasks.getValue()) {
-				task.trigger(now);
+	
+			// now do the actual firing.
+			for (Map.Entry<Long, List<Triggerable>> tasks: toRun) {
+				long now = tasks.getKey();
+				for (Triggerable task: tasks.getValue()) {
+					task.trigger(now);
+				}
 			}
 		}
 	}
@@ -80,6 +83,9 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 		if (isTerminated) {
 			throw new IllegalStateException("terminated");
 		}
+		if (isQuiesced) {
+			return new DummyFuture();
+		}
 
 		if (timestamp <= currentTime) {
 			try {
@@ -88,7 +94,6 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 				throw new RuntimeException(e);
 			}
 		}
-
 		List<Triggerable> tasks = registeredTasks.get(timestamp);
 		if (tasks == null) {
 			tasks = new ArrayList<>();
@@ -105,8 +110,16 @@ public class TestTimeServiceProvider extends TimeServiceProvider {
 	}
 
 	@Override
-	public void shutdownService() throws Exception {
-		isTerminated = true;
+	public void quiesceAndAwaitPending() {
+		if (!isTerminated) {
+			isQuiesced = true;
+			registeredTasks.clear();
+		}
+	}
+
+	@Override
+	public void shutdownService() {
+		this.isTerminated = true;
 	}
 
 	public int getNumRegisteredTimers() {

http://git-wip-us.apache.org/repos/asf/flink/blob/8aea8c8f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimeServiceProvider.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimeServiceProvider.java
index 42a4fa4..afa6f35 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimeServiceProvider.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimeServiceProvider.java
@@ -14,32 +14,70 @@
  * 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.runtime.operators.Triggerable;
+
 import java.util.concurrent.ScheduledFuture;
 
 /**
  * Defines the current processing time and handles all related actions,
  * such as register timers for tasks to be executed in the future.
+ * 
+ * <p>The access to the time via {@link #getCurrentProcessingTime()} is always available, regardless of
+ * whether the timer service has been shut down.
+ * 
+ * <p>The registration of timers follows a life cycle of three phases:
+ * <ol>
+ *     <li>In the initial state, it accepts timer registrations and triggers when the time is reached.</li>
+ *     <li>After calling {@link #quiesceAndAwaitPending()}, further calls to
+ *         {@link #registerTimer(long, Triggerable)} will not register any further timers, and will
+ *         return a "dummy" future as a result. This is used for clean shutdown, where currently firing
+ *         timers are waited for and no future timers can be scheduled, without causing hard exceptions.</li>
+ *     <li>After a call to {@link #shutdownService()}, all calls to {@link #registerTimer(long, Triggerable)}
+ *         will result in a hard exception.</li>
+ * </ol>
  */
 public abstract class TimeServiceProvider {
 
-	/** Returns the current processing time. */
+	/**
+	 * Returns the current processing time.
+	 */
 	public abstract long getCurrentProcessingTime();
 
-	/** Registers a task to be executed when (processing) time is {@code timestamp}.
-	 * @param timestamp
-	 * 						when the task is to be executed (in processing time)
-	 * @param target
-	 * 						the task to be executed
-	 * @return the result to be returned.
+	/**
+	 * Registers a task to be executed when (processing) time is {@code timestamp}.
+	 * 
+	 * @param timestamp   Time when the task is to be executed (in processing time)
+	 * @param target      The task to be executed
+	 * 
+	 * @return The future that represents the scheduled task. This always returns some future,
+	 *         even if the timer was shut down
 	 */
-	public abstract ScheduledFuture<?> registerTimer(final long timestamp, final Triggerable target);
+	public abstract ScheduledFuture<?> registerTimer(long timestamp, Triggerable target);
 
-	/** Returns <tt>true</tt> if the service has been shut down, <tt>false</tt> otherwise. */
+	/**
+	 * Returns <tt>true</tt> if the service has been shut down, <tt>false</tt> otherwise.
+	 */
 	public abstract boolean isTerminated();
 
-	/** Shuts down and clean up the timer service provider. */
-	public abstract void shutdownService() throws Exception;
+	/**
+	 * This method puts the service into a state where it does not register new timers, but
+	 * returns for each call to {@link #registerTimer(long, Triggerable)} only a "mock" future.
+	 * Furthermore, the method clears all not yet started timers, and awaits the completion
+	 * of currently executing timers.
+	 * 
+	 * <p>This method can be used to cleanly shut down the timer service. The using components
+	 * will not notice that the service is shut down (as for example via exceptions when registering
+	 * a new timer), but the service will simply not fire any timer any more.
+	 */
+	public abstract void quiesceAndAwaitPending() throws InterruptedException;
+
+	/**
+	 * Shuts down and clean up the timer service provider hard and immediately. This does not wait
+	 * for any timer to complete. Any further call to {@link #registerTimer(long, Triggerable)}
+	 * will result in a hard exception.
+	 */
+	public abstract void shutdownService();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8aea8c8f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java
index 16e658e..d0c5050 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/NoOpTimerService.java
@@ -26,7 +26,7 @@ import java.util.concurrent.ScheduledFuture;
 class NoOpTimerService extends TimeServiceProvider {
 
 	private volatile boolean terminated;
-	
+
 	@Override
 	public long getCurrentProcessingTime() {
 		return System.currentTimeMillis();
@@ -43,7 +43,10 @@ class NoOpTimerService extends TimeServiceProvider {
 	}
 
 	@Override
-	public void shutdownService() throws Exception {
+	public void quiesceAndAwaitPending() {}
+
+	@Override
+	public void shutdownService() {
 		terminated = true;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8aea8c8f/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java
new file mode 100644
index 0000000..ae895b6
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.core.testutils.OneShotLatch;
+import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
+
+import org.junit.Test;
+
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+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;
+
+public class DefaultTimeServiceProviderTest {
+
+	@Test
+	public void testImmediateShutdown() throws Exception {
+
+		final Object lock = new Object();
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+
+		final DefaultTimeServiceProvider timer = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(errorRef), lock);
+
+		try {
+			assertFalse(timer.isTerminated());
+
+			final OneShotLatch latch = new OneShotLatch();
+
+			// the task should trigger immediately and should block until terminated with interruption
+			timer.registerTimer(System.currentTimeMillis(), new Triggerable() {
+				@Override
+				public void trigger(long timestamp) throws Exception {
+					latch.trigger();
+					Thread.sleep(100000000);
+				}
+			});
+
+			latch.await();
+			timer.shutdownService();
+
+			// can only enter this scope after the triggerable is interrupted
+			//noinspection SynchronizationOnLocalVariableOrMethodParameter
+			synchronized (lock) {
+				assertTrue(timer.isTerminated());
+			}
+
+			try {
+				timer.registerTimer(System.currentTimeMillis() + 1000, new Triggerable() {
+					@Override
+					public void trigger(long timestamp) {}
+				});
+
+				fail("should result in an exception");
+			}
+			catch (IllegalStateException e) {
+				// expected
+			}
+
+			// obviously, we have an asynchronous interrupted exception
+			assertNotNull(errorRef.get());
+			assertTrue(errorRef.get().getCause() instanceof InterruptedException);
+
+			assertEquals(0, timer.getNumTasksScheduled());
+		}
+		finally {
+			timer.shutdownService();
+		}
+	}
+
+	@Test
+	public void testQuiescing() throws Exception {
+
+		final Object lock = new Object();
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+
+		final DefaultTimeServiceProvider timer = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(errorRef), lock);
+
+		try {
+			final OneShotLatch latch = new OneShotLatch();
+
+			final ReentrantLock scopeLock = new ReentrantLock();
+
+			timer.registerTimer(System.currentTimeMillis() + 20, new Triggerable() {
+				@Override
+				public void trigger(long timestamp) throws Exception {
+					scopeLock.lock();
+					try {
+						latch.trigger();
+						// delay a bit before leaving the method
+						Thread.sleep(5);
+					} finally {
+						scopeLock.unlock();
+					}
+				}
+			});
+
+			// after the task triggered, shut the timer down cleanly, waiting for the task to finish
+			latch.await();
+			timer.quiesceAndAwaitPending();
+
+			// should be able to immediately acquire the lock, since the task must have exited by now 
+			assertTrue(scopeLock.tryLock());
+
+			// should be able to schedule more tasks (that never get executed)
+			ScheduledFuture<?> future = timer.registerTimer(System.currentTimeMillis() - 5, new Triggerable() {
+				@Override
+				public void trigger(long timestamp) throws Exception {
+					throw new Exception("test");
+				}
+			});
+			assertNotNull(future);
+
+			// nothing should be scheduled right now
+			assertEquals(0, timer.getNumTasksScheduled());
+
+			// check that no asynchronous error was reported - that ensures that the newly scheduled 
+			// triggerable did, in fact, not trigger
+			if (errorRef.get() != null) {
+				throw new Exception(errorRef.get());
+			}
+		}
+		finally {
+			timer.shutdownService();
+		}
+	}
+
+	@Test
+	public void testFutureCancellation() throws Exception {
+
+		final Object lock = new Object();
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+
+		final DefaultTimeServiceProvider timer = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(errorRef), lock);
+
+		try {
+			assertEquals(0, timer.getNumTasksScheduled());
+
+			// schedule something
+			ScheduledFuture<?> future = timer.registerTimer(System.currentTimeMillis() + 100000000, new Triggerable() {
+				@Override
+				public void trigger(long timestamp) {}
+			});
+			assertEquals(1, timer.getNumTasksScheduled());
+
+			future.cancel(false);
+
+			assertEquals(0, timer.getNumTasksScheduled());
+		}
+		finally {
+			timer.shutdownService();
+		}
+	}
+}


[07/17] flink git commit: [FLINK-4737] [core] Add support for bz2 and xy compression in flink-core.

Posted by se...@apache.org.
[FLINK-4737] [core] Add support for bz2 and xy compression in flink-core.

Adds a dependency on 'commons-compression'.

This closes #2002


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

Branch: refs/heads/master
Commit: 81aec4109adc18f4bd4d1ddb91892faed10f4f14
Parents: 954ef08
Author: Milosz Tanski <mt...@gmail.com>
Authored: Wed May 18 00:04:36 2016 -0400
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:13 2016 +0200

----------------------------------------------------------------------
 docs/dev/batch/index.md                         | 10 ++++
 flink-core/pom.xml                              | 10 +++-
 .../flink/api/common/io/FileInputFormat.java    |  6 ++-
 .../io/compression/Bzip2InputStreamFactory.java | 50 ++++++++++++++++++++
 .../compression/InflaterInputStreamFactory.java |  3 +-
 .../io/compression/XZInputStreamFactory.java    | 49 +++++++++++++++++++
 6 files changed, 123 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/docs/dev/batch/index.md
----------------------------------------------------------------------
diff --git a/docs/dev/batch/index.md b/docs/dev/batch/index.md
index 5cdc36d..0b1c9f9 100644
--- a/docs/dev/batch/index.md
+++ b/docs/dev/batch/index.md
@@ -1114,6 +1114,16 @@ The following table lists the currently supported compression methods.
       <td><code>.gz</code>, <code>.gzip</code></td>
       <td>no</td>
     </tr>
+    <tr>
+      <td><strong>Bzip2</strong></td>
+      <td><code>.bz2</code></td>
+      <td>no</td>
+    </tr>
+    <tr>
+      <td><strong>XZ</strong></td>
+      <td><code>.xz</code></td>
+      <td>no</td>
+    </tr>
   </tbody>
 </table>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index fe24f0e..40e5a2e 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -75,9 +75,15 @@ under the License.
 			<groupId>commons-collections</groupId>
 			<artifactId>commons-collections</artifactId>
 		</dependency>
-		
-		<!-- test dependencies -->
 
+		<!-- Commons compression, for additional decompressors -->
+		<dependency>
+			<groupId>org.apache.commons</groupId>
+			<artifactId>commons-compress</artifactId>
+			<version>1.4</version>
+		</dependency>
+
+		<!-- test dependencies -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-test-utils-junit</artifactId>

http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
index d0f5166..1d092af 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
@@ -19,9 +19,11 @@
 package org.apache.flink.api.common.io;
 
 import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.io.compression.Bzip2InputStreamFactory;
 import org.apache.flink.api.common.io.compression.DeflateInflaterInputStreamFactory;
 import org.apache.flink.api.common.io.compression.GzipInflaterInputStreamFactory;
 import org.apache.flink.api.common.io.compression.InflaterInputStreamFactory;
+import org.apache.flink.api.common.io.compression.XZInputStreamFactory;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -116,7 +118,9 @@ public abstract class FileInputFormat<OT> extends RichInputFormat<OT, FileInputS
 	private static void initDefaultInflaterInputStreamFactories() {
 		InflaterInputStreamFactory<?>[] defaultFactories = {
 				DeflateInflaterInputStreamFactory.getInstance(),
-				GzipInflaterInputStreamFactory.getInstance()
+				GzipInflaterInputStreamFactory.getInstance(),
+				Bzip2InputStreamFactory.getInstance(),
+				XZInputStreamFactory.getInstance(),
 		};
 		for (InflaterInputStreamFactory<?> inputStreamFactory : defaultFactories) {
 			for (String fileExtension : inputStreamFactory.getCommonFileExtensions()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/src/main/java/org/apache/flink/api/common/io/compression/Bzip2InputStreamFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/compression/Bzip2InputStreamFactory.java b/flink-core/src/main/java/org/apache/flink/api/common/io/compression/Bzip2InputStreamFactory.java
new file mode 100644
index 0000000..d204907
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/compression/Bzip2InputStreamFactory.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.api.common.io.compression;
+
+import org.apache.flink.annotation.Internal;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
+
+@Internal
+public class Bzip2InputStreamFactory implements InflaterInputStreamFactory<BZip2CompressorInputStream> {
+
+	private static Bzip2InputStreamFactory INSTANCE = null;
+
+	public static Bzip2InputStreamFactory getInstance() {
+		if (INSTANCE == null) {
+			INSTANCE = new Bzip2InputStreamFactory();
+		}
+		return INSTANCE;
+	}
+
+	@Override
+	public BZip2CompressorInputStream create(InputStream in) throws IOException {
+		return new BZip2CompressorInputStream(in);
+	}
+
+	@Override
+	public Collection<String> getCommonFileExtensions() {
+		return Collections.singleton("bz2");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/src/main/java/org/apache/flink/api/common/io/compression/InflaterInputStreamFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/compression/InflaterInputStreamFactory.java b/flink-core/src/main/java/org/apache/flink/api/common/io/compression/InflaterInputStreamFactory.java
index 7fbc50d..c973763 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/compression/InflaterInputStreamFactory.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/compression/InflaterInputStreamFactory.java
@@ -23,13 +23,12 @@ import org.apache.flink.annotation.Internal;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.Collection;
-import java.util.zip.InflaterInputStream;
 
 /**
  * Creates a new instance of a certain subclass of {@link java.util.zip.InflaterInputStream}.
  */
 @Internal
-public interface InflaterInputStreamFactory<T extends InflaterInputStream> {
+public interface InflaterInputStreamFactory<T extends InputStream> {
 
 	/**
 	 * Creates a {@link java.util.zip.InflaterInputStream} that wraps the given input stream.

http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/src/main/java/org/apache/flink/api/common/io/compression/XZInputStreamFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/compression/XZInputStreamFactory.java b/flink-core/src/main/java/org/apache/flink/api/common/io/compression/XZInputStreamFactory.java
new file mode 100644
index 0000000..c80de40
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/compression/XZInputStreamFactory.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.api.common.io.compression;
+
+import org.apache.commons.compress.compressors.xz.XZCompressorInputStream;
+import org.apache.flink.annotation.Internal;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Collection;
+import java.util.Collections;
+
+@Internal
+public class XZInputStreamFactory implements InflaterInputStreamFactory<XZCompressorInputStream> {
+
+	private static XZInputStreamFactory INSTANCE = null;
+
+	public static XZInputStreamFactory getInstance() {
+		if (INSTANCE == null) {
+			INSTANCE = new XZInputStreamFactory();
+		}
+		return INSTANCE;
+	}
+
+	@Override
+	public XZCompressorInputStream create(InputStream in) throws IOException {
+		return new XZCompressorInputStream(in, true);
+	}
+
+	@Override
+	public Collection<String> getCommonFileExtensions() {
+		return Collections.singleton("xz");
+	}
+}


[06/17] flink git commit: [FLINK-4329] [streaming api] Fix Streaming File Source Timestamps/Watermarks Handling

Posted by se...@apache.org.
[FLINK-4329] [streaming api] Fix Streaming File Source Timestamps/Watermarks Handling

This closes #2546


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

Branch: refs/heads/master
Commit: 8ff451bec58e9f5800eb77c74c1d7457b776cc94
Parents: c62776f
Author: kl0u <kk...@gmail.com>
Authored: Thu Aug 25 17:38:49 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:13 2016 +0200

----------------------------------------------------------------------
 .../state/RocksDBAsyncSnapshotTest.java         |  11 +-
 .../ContinuousFileMonitoringFunctionITCase.java |  17 +-
 .../hdfstests/ContinuousFileMonitoringTest.java | 209 ++++++++++++--
 .../fs/bucketing/BucketingSinkTest.java         |   4 +-
 .../source/ContinuousFileReaderOperator.java    |  96 ++++---
 .../streaming/api/operators/StreamSource.java   | 275 +-----------------
 .../api/operators/StreamSourceContexts.java     | 284 +++++++++++++++++++
 .../runtime/tasks/AsyncExceptionHandler.java    |   8 +-
 .../tasks/DefaultTimeServiceProvider.java       |  11 +-
 .../runtime/tasks/OneInputStreamTask.java       |   2 +-
 .../streaming/runtime/tasks/StreamTask.java     |  54 +---
 .../runtime/tasks/TwoInputStreamTask.java       |   2 +-
 .../operators/StreamSourceOperatorTest.java     |  17 +-
 .../runtime/operators/TimeProviderTest.java     |  79 ++++--
 ...AlignedProcessingTimeWindowOperatorTest.java |  34 ++-
 ...AlignedProcessingTimeWindowOperatorTest.java |  36 ++-
 .../runtime/tasks/StreamMockEnvironment.java    |   8 +-
 .../KeyedOneInputStreamOperatorTestHarness.java |   4 +-
 .../util/OneInputStreamOperatorTestHarness.java |  23 +-
 19 files changed, 694 insertions(+), 480 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
index bccbabc..2ebd84a 100644
--- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
+++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java
@@ -86,7 +86,7 @@ public class RocksDBAsyncSnapshotTest {
 	}
 
 	/**
-	 * This ensures that asynchronous state handles are actually materialized asynchonously.
+	 * This ensures that asynchronous state handles are actually materialized asynchronously.
 	 *
 	 * <p>We use latches to block at various stages and see if the code still continues through
 	 * the parts that are not asynchronous. If the checkpoint is not done asynchronously the
@@ -168,7 +168,6 @@ public class RocksDBAsyncSnapshotTest {
 				while (!field.getBoolean(task)) {
 					Thread.sleep(10);
 				}
-
 			}
 		}
 
@@ -189,7 +188,9 @@ public class RocksDBAsyncSnapshotTest {
 		Assert.assertTrue(threadPool.awaitTermination(60_000, TimeUnit.MILLISECONDS));
 
 		testHarness.waitForTaskCompletion();
-		task.checkTimerException();
+		if (mockEnv.wasFailedExternally()) {
+			Assert.fail("Unexpected exception during execution.");
+		}
 	}
 
 	/**
@@ -261,8 +262,10 @@ public class RocksDBAsyncSnapshotTest {
 			threadPool.shutdown();
 			Assert.assertTrue(threadPool.awaitTermination(60_000, TimeUnit.MILLISECONDS));
 			testHarness.waitForTaskCompletion();
-			task.checkTimerException();
 
+			if (mockEnv.wasFailedExternally()) {
+				throw new AsynchronousException(new InterruptedException("Exception was thrown as expected."));
+			}
 			Assert.fail("Operation completed. Cancel failed.");
 		} catch (Exception expected) {
 			AsynchronousException asynchronousException = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringFunctionITCase.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringFunctionITCase.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringFunctionITCase.java
index 663345c..079bf04 100644
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringFunctionITCase.java
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringFunctionITCase.java
@@ -120,7 +120,7 @@ public class ContinuousFileMonitoringFunctionITCase extends StreamingProgramTest
 
 		try {
 			StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-			env.setParallelism(1);
+			env.setParallelism(4);
 
 			format.setFilesFilter(FilePathFilter.createDefaultFilter());
 			ContinuousFileMonitoringFunction<String> monitoringFunction =
@@ -130,7 +130,7 @@ public class ContinuousFileMonitoringFunctionITCase extends StreamingProgramTest
 
 			TypeInformation<String> typeInfo = TypeExtractor.getInputFormatTypes(format);
 			ContinuousFileReaderOperator<String, ?> reader = new ContinuousFileReaderOperator<>(format);
-			TestingSinkFunction sink = new TestingSinkFunction(monitoringFunction);
+			TestingSinkFunction sink = new TestingSinkFunction();
 
 			DataStream<FileInputSplit> splits = env.addSource(monitoringFunction);
 			splits.transform("FileSplitReader", typeInfo, reader).addSink(sink).setParallelism(1);
@@ -161,16 +161,10 @@ public class ContinuousFileMonitoringFunctionITCase extends StreamingProgramTest
 
 	private static class TestingSinkFunction extends RichSinkFunction<String> {
 
-		private final ContinuousFileMonitoringFunction src;
-
 		private int elementCounter = 0;
 		private Map<Integer, Integer> elementCounters = new HashMap<>();
 		private Map<Integer, List<String>> collectedContent = new HashMap<>();
 
-		TestingSinkFunction(ContinuousFileMonitoringFunction monitoringFunction) {
-			this.src = monitoringFunction;
-		}
-
 		@Override
 		public void open(Configuration parameters) throws Exception {
 			// this sink can only work with DOP 1
@@ -200,13 +194,6 @@ public class ContinuousFileMonitoringFunctionITCase extends StreamingProgramTest
 				Assert.assertEquals(cntntStr.toString(), expectedContents.get(fileIdx));
 			}
 			expectedContents.clear();
-
-			src.cancel();
-			try {
-				src.close();
-			} catch (Exception e) {
-				e.printStackTrace();
-			}
 		}
 
 		private int getLineNo(String line) {

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java
index 8a700f5..36b5c5e 100644
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java
@@ -27,12 +27,14 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.api.common.io.FilePathFilter;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
 import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
 import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
 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 org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileUtil;
@@ -51,7 +53,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Queue;
 import java.util.Set;
 
 public class ContinuousFileMonitoringTest {
@@ -106,6 +107,155 @@ public class ContinuousFileMonitoringTest {
 	//						TESTS
 
 	@Test
+	public void testFileReadingOperatorWithIngestionTime() throws Exception {
+		Set<org.apache.hadoop.fs.Path> filesCreated = new HashSet<>();
+		Map<Integer, String> expectedFileContents = new HashMap<>();
+
+		for(int i = 0; i < NO_OF_FILES; i++) {
+			Tuple2<org.apache.hadoop.fs.Path, String> file = fillWithData(hdfsURI, "file", i, "This is test line.");
+			filesCreated.add(file.f0);
+			expectedFileContents.put(i, file.f1);
+		}
+
+		TextInputFormat format = new TextInputFormat(new Path(hdfsURI));
+		TypeInformation<String> typeInfo = TypeExtractor.getInputFormatTypes(format);
+
+		final long watermarkInterval = 10;
+		ExecutionConfig executionConfig = new ExecutionConfig();
+		executionConfig.setAutoWatermarkInterval(watermarkInterval);
+
+		ContinuousFileReaderOperator<String, ?> reader = new ContinuousFileReaderOperator<>(format);
+		reader.setOutputType(typeInfo, executionConfig);
+
+		final TestTimeServiceProvider timeServiceProvider = new TestTimeServiceProvider();
+		final OneInputStreamOperatorTestHarness<FileInputSplit, String> tester =
+			new OneInputStreamOperatorTestHarness<>(reader, executionConfig, timeServiceProvider);
+		tester.setTimeCharacteristic(TimeCharacteristic.IngestionTime);
+		tester.open();
+
+		Assert.assertEquals(TimeCharacteristic.IngestionTime, tester.getTimeCharacteristic());
+
+		// test that watermarks are correctly emitted
+
+		timeServiceProvider.setCurrentTime(201);
+		timeServiceProvider.setCurrentTime(301);
+		timeServiceProvider.setCurrentTime(401);
+		timeServiceProvider.setCurrentTime(501);
+
+		int i = 0;
+		for(Object line: tester.getOutput()) {
+			if (!(line instanceof Watermark)) {
+				Assert.fail("Only watermarks are expected here ");
+			}
+			Watermark w = (Watermark) line;
+			Assert.assertEquals(200 + (i * 100), w.getTimestamp());
+			i++;
+		}
+
+		// clear the output to get the elements only and the final watermark
+		tester.getOutput().clear();
+		Assert.assertEquals(0, tester.getOutput().size());
+
+		// create the necessary splits for the test
+		FileInputSplit[] splits = format.createInputSplits(
+			reader.getRuntimeContext().getNumberOfParallelSubtasks());
+
+		// and feed them to the operator
+		Map<Integer, List<String>> actualFileContents = new HashMap<>();
+
+		long lastSeenWatermark = Long.MIN_VALUE;
+		int lineCounter = 0;	// counter for the lines read from the splits
+		int watermarkCounter = 0;
+
+		for(FileInputSplit split: splits) {
+
+			// set the next "current processing time".
+			long nextTimestamp = timeServiceProvider.getCurrentProcessingTime() + watermarkInterval;
+			timeServiceProvider.setCurrentTime(nextTimestamp);
+
+			// send the next split to be read and wait until it is fully read.
+			tester.processElement(new StreamRecord<>(split));
+			synchronized (tester.getCheckpointLock()) {
+				while (tester.getOutput().isEmpty() || tester.getOutput().size() != (LINES_PER_FILE + 1)) {
+					tester.getCheckpointLock().wait(10);
+				}
+			}
+
+			// verify that the results are the expected
+			for(Object line: tester.getOutput()) {
+				if (line instanceof StreamRecord) {
+					StreamRecord<String> element = (StreamRecord<String>) line;
+					lineCounter++;
+
+					Assert.assertEquals(nextTimestamp, element.getTimestamp());
+
+					int fileIdx = Character.getNumericValue(element.getValue().charAt(0));
+					List<String> content = actualFileContents.get(fileIdx);
+					if (content == null) {
+						content = new ArrayList<>();
+						actualFileContents.put(fileIdx, content);
+					}
+					content.add(element.getValue() + "\n");
+				} else if (line instanceof Watermark) {
+					long watermark = ((Watermark) line).getTimestamp();
+
+					Assert.assertEquals(nextTimestamp - (nextTimestamp % watermarkInterval), watermark);
+					Assert.assertTrue(watermark > lastSeenWatermark);
+					watermarkCounter++;
+
+					lastSeenWatermark = watermark;
+				} else {
+					Assert.fail("Unknown element in the list.");
+				}
+			}
+
+			// clean the output to be ready for the next split
+			tester.getOutput().clear();
+		}
+
+		// now we are processing one split after the other,
+		// so all the elements must be here by now.
+		Assert.assertEquals(NO_OF_FILES * LINES_PER_FILE, lineCounter);
+
+		// because we expect one watermark per split.
+		Assert.assertEquals(NO_OF_FILES, watermarkCounter);
+
+		// then close the reader gracefully so that the Long.MAX watermark is emitted
+		synchronized (tester.getCheckpointLock()) {
+			tester.close();
+		}
+
+		for(org.apache.hadoop.fs.Path file: filesCreated) {
+			hdfs.delete(file, false);
+		}
+
+		// check if the last element is the LongMax watermark (by now this must be the only element)
+		Assert.assertEquals(1, tester.getOutput().size());
+		Assert.assertTrue(tester.getOutput().peek() instanceof Watermark);
+		Assert.assertEquals(Long.MAX_VALUE, ((Watermark) tester.getOutput().poll()).getTimestamp());
+
+		// check if the elements are the expected ones.
+		Assert.assertEquals(expectedFileContents.size(), actualFileContents.size());
+		for (Integer fileIdx: expectedFileContents.keySet()) {
+			Assert.assertTrue("file" + fileIdx + " not found", actualFileContents.keySet().contains(fileIdx));
+
+			List<String> cntnt = actualFileContents.get(fileIdx);
+			Collections.sort(cntnt, new Comparator<String>() {
+				@Override
+				public int compare(String o1, String o2) {
+					return getLineNo(o1) - getLineNo(o2);
+				}
+			});
+
+			StringBuilder cntntStr = new StringBuilder();
+			for (String line: cntnt) {
+				cntntStr.append(line);
+			}
+			Assert.assertEquals(expectedFileContents.get(fileIdx), cntntStr.toString());
+		}
+	}
+
+	@Test
 	public void testFileReadingOperator() throws Exception {
 		Set<org.apache.hadoop.fs.Path> filesCreated = new HashSet<>();
 		Map<Integer, String> expectedFileContents = new HashMap<>();
@@ -119,10 +269,11 @@ public class ContinuousFileMonitoringTest {
 		TypeInformation<String> typeInfo = TypeExtractor.getInputFormatTypes(format);
 
 		ContinuousFileReaderOperator<String, ?> reader = new ContinuousFileReaderOperator<>(format);
+		reader.setOutputType(typeInfo, new ExecutionConfig());
+
 		OneInputStreamOperatorTestHarness<FileInputSplit, String> tester =
 			new OneInputStreamOperatorTestHarness<>(reader);
-
-		reader.setOutputType(typeInfo, new ExecutionConfig());
+		tester.setTimeCharacteristic(TimeCharacteristic.EventTime);
 		tester.open();
 
 		// create the necessary splits for the test
@@ -134,38 +285,38 @@ public class ContinuousFileMonitoringTest {
 			tester.processElement(new StreamRecord<>(split));
 		}
 
-		// then close the reader gracefully
+		// then close the reader gracefully (and wait to finish reading)
 		synchronized (tester.getCheckpointLock()) {
 			tester.close();
 		}
 
-		/*
-		* Given that the reader is multithreaded, the test finishes before the reader thread finishes
-		* reading. This results in files being deleted by the test before being read, thus throwing an exception.
-		* In addition, even if file deletion happens at the end, the results are not ready for testing.
-		* To face this, we wait until all the output is collected or until the waiting time exceeds 1000 ms, or 1s.
-		*/
+		// the lines received must be the elements in the files +1 for for the longMax watermark
+		// we are in event time, which emits no watermarks, so the last watermark will mark the
+		// of the input stream.
 
-		long start = System.currentTimeMillis();
-		Queue<Object> output;
-		do {
-			output = tester.getOutput();
-			Thread.sleep(50);
-		} while ((output == null || output.size() != NO_OF_FILES * LINES_PER_FILE) && (System.currentTimeMillis() - start) < 1000);
+		Assert.assertEquals(NO_OF_FILES * LINES_PER_FILE + 1, tester.getOutput().size());
 
 		Map<Integer, List<String>> actualFileContents = new HashMap<>();
+		Object lastElement = null;
 		for(Object line: tester.getOutput()) {
-			StreamRecord<String> element = (StreamRecord<String>) line;
-
-			int fileIdx = Character.getNumericValue(element.getValue().charAt(0));
-			List<String> content = actualFileContents.get(fileIdx);
-			if(content == null) {
-				content = new ArrayList<>();
-				actualFileContents.put(fileIdx, content);
+			lastElement = line;
+			if (line instanceof StreamRecord) {
+				StreamRecord<String> element = (StreamRecord<String>) line;
+
+				int fileIdx = Character.getNumericValue(element.getValue().charAt(0));
+				List<String> content = actualFileContents.get(fileIdx);
+				if (content == null) {
+					content = new ArrayList<>();
+					actualFileContents.put(fileIdx, content);
+				}
+				content.add(element.getValue() + "\n");
 			}
-			content.add(element.getValue() +"\n");
 		}
 
+		// check if the last element is the LongMax watermark
+		Assert.assertTrue(lastElement instanceof Watermark);
+		Assert.assertEquals(Long.MAX_VALUE, ((Watermark) lastElement).getTimestamp());
+
 		Assert.assertEquals(expectedFileContents.size(), actualFileContents.size());
 		for (Integer fileIdx: expectedFileContents.keySet()) {
 			Assert.assertTrue("file" + fileIdx + " not found", actualFileContents.keySet().contains(fileIdx));
@@ -224,7 +375,7 @@ public class ContinuousFileMonitoringTest {
 		monitoringFunction.open(new Configuration());
 		monitoringFunction.run(new TestingSourceContext(monitoringFunction, uniqFilesFound));
 
-		Assert.assertTrue(uniqFilesFound.size() == NO_OF_FILES);
+		Assert.assertEquals(NO_OF_FILES, uniqFilesFound.size());
 		for(int i = 0; i < NO_OF_FILES; i++) {
 			org.apache.hadoop.fs.Path file = new org.apache.hadoop.fs.Path(hdfsURI + "/file" + i);
 			Assert.assertTrue(uniqFilesFound.contains(file.toString()));
@@ -268,8 +419,8 @@ public class ContinuousFileMonitoringTest {
 		t.interrupt();
 		fc.join();
 
-		Assert.assertTrue(fc.getFilesCreated().size() == NO_OF_FILES);
-		Assert.assertTrue(uniqFilesFound.size() == NO_OF_FILES);
+		Assert.assertEquals(NO_OF_FILES, fc.getFilesCreated().size());
+		Assert.assertEquals(NO_OF_FILES, uniqFilesFound.size());
 
 		Set<org.apache.hadoop.fs.Path> filesCreated = fc.getFilesCreated();
 		Set<String> fileNamesCreated = new HashSet<>();
@@ -316,7 +467,7 @@ public class ContinuousFileMonitoringTest {
 		// wait until all the files are created
 		fc.join();
 
-		Assert.assertTrue(filesCreated.size() == NO_OF_FILES);
+		Assert.assertEquals(NO_OF_FILES, filesCreated.size());
 
 		Set<String> fileNamesCreated = new HashSet<>();
 		for (org.apache.hadoop.fs.Path path: fc.getFilesCreated()) {
@@ -337,7 +488,7 @@ public class ContinuousFileMonitoringTest {
 
 	private int getLineNo(String line) {
 		String[] tkns = line.split("\\s");
-		Assert.assertTrue(tkns.length == 6);
+		Assert.assertEquals(6, tkns.length);
 		return Integer.parseInt(tkns[tkns.length - 1]);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
index e274fdd..ac1e3f0 100644
--- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java
@@ -70,7 +70,7 @@ public class BucketingSinkTest {
 	private static org.apache.hadoop.fs.FileSystem dfs;
 	private static String hdfsURI;
 
-	private OneInputStreamOperatorTestHarness<String, Object> createTestSink(File dataDir, TimeServiceProvider clock) {
+	private OneInputStreamOperatorTestHarness<String, Object> createTestSink(File dataDir, TestTimeServiceProvider clock) {
 		BucketingSink<String> sink = new BucketingSink<String>(dataDir.getAbsolutePath())
 			.setBucketer(new Bucketer<String>() {
 				private static final long serialVersionUID = 1L;
@@ -91,7 +91,7 @@ public class BucketingSinkTest {
 	}
 
 	private <T> OneInputStreamOperatorTestHarness<T, Object> createTestSink(BucketingSink<T> sink,
-																			TimeServiceProvider clock) {
+																			TestTimeServiceProvider clock) {
 		return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), new ExecutionConfig(), clock);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
index 838bee6..35e72a7 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java
@@ -29,14 +29,13 @@ import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.metrics.Counter;
 import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 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.operators.StreamSourceContexts;
 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.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,20 +43,24 @@ import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 import java.io.Serializable;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Queue;
 
+import static org.apache.flink.util.Preconditions.checkState;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * This is the operator that reads the {@link FileInputSplit FileInputSplits} received from
- * the preceding {@link ContinuousFileMonitoringFunction}. This operator will receive just the split descriptors
- * and then read and emit records. This may lead to increased backpressure. To avoid this, we have another
- * thread ({@link SplitReader}) actually reading the splits and emitting the elements, which is separate from
- * the thread forwarding the checkpoint barriers. The two threads sync on the {@link StreamTask#getCheckpointLock()}
- * so that the checkpoints reflect the current state.
+ * the preceding {@link ContinuousFileMonitoringFunction}. This operator can have parallelism
+ * greater than 1, contrary to the {@link ContinuousFileMonitoringFunction} which has
+ * a parallelism of 1.
+ * <p/>
+ * This operator will receive the split descriptors, put them in a queue, and have another
+ * thread read the actual data from the split. This architecture allows the separation of the
+ * reading thread, from the one emitting the checkpoint barriers, thus removing any potential
+ * back-pressure.
  */
 @Internal
 public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends AbstractStreamOperator<OUT>
@@ -67,16 +70,16 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 
 	private static final Logger LOG = LoggerFactory.getLogger(ContinuousFileReaderOperator.class);
 
+	/** A value that serves as a kill-pill to stop the reading thread when no more splits remain. */
 	private static final FileInputSplit EOS = new FileInputSplit(-1, null, -1, -1, null);
 
-	private transient SplitReader<S, OUT> reader;
-	private transient TimestampedCollector<OUT> collector;
-
 	private FileInputFormat<OUT> format;
 	private TypeSerializer<OUT> serializer;
 
 	private transient Object checkpointLock;
 
+	private transient SplitReader<S, OUT> reader;
+	private transient SourceFunction.SourceContext<OUT> readerContext;
 	private Tuple3<List<FileInputSplit>, FileInputSplit, S> readerState;
 
 	public ContinuousFileReaderOperator(FileInputFormat<OUT> format) {
@@ -92,25 +95,22 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 	public void open() throws Exception {
 		super.open();
 
-		if (this.serializer == null) {
-			throw new IllegalStateException("The serializer has not been set. " +
-				"Probably the setOutputType() was not called and this should not have happened. " +
-				"Please report it.");
-		}
+		checkState(this.reader == null, "The reader is already initialized.");
+		checkState(this.serializer != null, "The serializer has not been set. " +
+			"Probably the setOutputType() was not called. Please report it.");
 
 		this.format.setRuntimeContext(getRuntimeContext());
 		this.format.configure(new Configuration());
-
-		this.collector = new TimestampedCollector<>(output);
 		this.checkpointLock = getContainingTask().getCheckpointLock();
 
-		Preconditions.checkState(reader == null, "The reader is already initialized.");
+		// set the reader context based on the time characteristic
+		final TimeCharacteristic timeCharacteristic = getOperatorConfig().getTimeCharacteristic();
+		final long watermarkInterval = getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval();
+		this.readerContext = StreamSourceContexts.getSourceContext(
+			timeCharacteristic, getTimerService(), checkpointLock, output, watermarkInterval);
 
-		this.reader = new SplitReader<>(format, serializer, collector, checkpointLock, readerState);
-
-		// the readerState is needed for the initialization of the reader
-		// when recovering from a failure. So after the initialization,
-		// we can set it to null.
+		// and initialize the split reading thread
+		this.reader = new SplitReader<>(format, serializer, readerContext, checkpointLock, readerState);
 		this.readerState = null;
 		this.reader.start();
 	}
@@ -122,7 +122,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 
 	@Override
 	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
+		// we do nothing because we emit our own watermarks if needed.
 	}
 
 	@Override
@@ -156,7 +156,8 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 			}
 		}
 		reader = null;
-		collector = null;
+		readerContext = null;
+		readerState = null;
 		format = null;
 		serializer = null;
 	}
@@ -177,7 +178,16 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 			// called by the StreamTask while having it.
 			checkpointLock.wait();
 		}
-		collector.close();
+
+		// finally if we are closed normally and we are operating on
+		// event or ingestion time, emit the max watermark indicating
+		// the end of the stream, like a normal source would do.
+
+		if (readerContext != null) {
+			readerContext.emitWatermark(Watermark.MAX_WATERMARK);
+			readerContext.close();
+		}
+		output.close();
 	}
 
 	private class SplitReader<S extends Serializable, OT> extends Thread {
@@ -188,7 +198,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 		private final TypeSerializer<OT> serializer;
 
 		private final Object checkpointLock;
-		private final TimestampedCollector<OT> collector;
+		private final SourceFunction.SourceContext<OT> readerContext;
 
 		private final Queue<FileInputSplit> pendingSplits;
 
@@ -200,16 +210,16 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 
 		private SplitReader(FileInputFormat<OT> format,
 					TypeSerializer<OT> serializer,
-					TimestampedCollector<OT> collector,
+					SourceFunction.SourceContext<OT> readerContext,
 					Object checkpointLock,
 					Tuple3<List<FileInputSplit>, FileInputSplit, S> restoredState) {
 
 			this.format = checkNotNull(format, "Unspecified FileInputFormat.");
 			this.serializer = checkNotNull(serializer, "Unspecified Serializer.");
+			this.readerContext = checkNotNull(readerContext, "Unspecified Reader Context.");
+			this.checkpointLock = checkNotNull(checkpointLock, "Unspecified checkpoint lock.");
 
-			this.pendingSplits = new LinkedList<>();
-			this.collector = collector;
-			this.checkpointLock = checkpointLock;
+			this.pendingSplits = new ArrayDeque<>();
 			this.isRunning = true;
 
 			// this is the case where a task recovers from a previous failed attempt
@@ -219,7 +229,6 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 				S formatState = restoredState.f2;
 
 				for (FileInputSplit split : pending) {
-					Preconditions.checkArgument(!pendingSplits.contains(split), "Duplicate split entry to read: " + split + ".");
 					pendingSplits.add(split);
 				}
 
@@ -229,9 +238,8 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 		}
 
 		private void addSplit(FileInputSplit split) {
-			Preconditions.checkNotNull(split);
+			checkNotNull(split, "Cannot insert a null value in the pending splits queue.");
 			synchronized (checkpointLock) {
-				Preconditions.checkArgument(!pendingSplits.contains(split), "Duplicate split entry to read: " + split + ".");
 				this.pendingSplits.add(split);
 			}
 		}
@@ -267,7 +275,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 								checkpointableFormat.reopen(currentSplit, restoredFormatState);
 							} else {
 								// this is the case of a non-checkpointable input format that will reprocess the last split.
-								LOG.info("Format " + this.format.getClass().getName() + " used is not checkpointable.");
+								LOG.info("Format " + this.format.getClass().getName() + " does not support checkpointing.");
 								format.open(currentSplit);
 							}
 							// reset the restored state to null for the next iteration
@@ -299,7 +307,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 							synchronized (checkpointLock) {
 								nextElement = format.nextRecord(nextElement);
 								if (nextElement != null) {
-									collector.collect(nextElement);
+									readerContext.collect(nextElement);
 								} else {
 									break;
 								}
@@ -318,10 +326,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 				}
 
 			} catch (Throwable e) {
-				if (isRunning) {
-					LOG.error("Caught exception processing split: ", currentSplit);
-				}
-				getContainingTask().failExternally(e);
+				getContainingTask().handleAsyncException("Caught exception when processing split: " + currentSplit, e);
 			} finally {
 				synchronized (checkpointLock) {
 					LOG.info("Reader terminated, and exiting...");
@@ -358,7 +363,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 							restoredFormatState;
 					return new Tuple3<>(snapshot, currentSplit, formatState);
 				} else {
-					LOG.info("The format used is not checkpointable. The current input split will be restarted upon recovery.");
+					LOG.info("The format does not support checkpointing. The current input split will be re-read from start upon recovery.");
 					return new Tuple3<>(snapshot, currentSplit, null);
 				}
 			} else {
@@ -404,7 +409,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 		FileInputSplit currSplit = (FileInputSplit) ois.readObject();
 
 		// read the pending splits list
-		List<FileInputSplit> pendingSplits = new LinkedList<>();
+		List<FileInputSplit> pendingSplits = new ArrayList<>();
 		int noOfSplits = ois.readInt();
 		for (int i = 0; i < noOfSplits; i++) {
 			FileInputSplit split = (FileInputSplit) ois.readObject();
@@ -416,8 +421,7 @@ public class ContinuousFileReaderOperator<OUT, S extends Serializable> extends A
 		S formatState = (S) ois.readObject();
 
 		// set the whole reader state for the open() to find.
-		Preconditions.checkState(this.readerState == null,
-			"The reader state has already been initialized.");
+		checkState(this.readerState == null, "The reader state has already been initialized.");
 
 		this.readerState = new Tuple3<>(pendingSplits, currSplit, formatState);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
index 22987ab..1409ae4 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
@@ -21,11 +21,7 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 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.runtime.tasks.TimeServiceProvider;
-
-import java.util.concurrent.ScheduledFuture;
 
 /**
  * {@link StreamOperator} for streaming sources.
@@ -57,26 +53,11 @@ public class StreamSource<OUT, SRC extends SourceFunction<OUT>>
 	
 	public void run(final Object lockingObject, final Output<StreamRecord<OUT>> collector) throws Exception {
 		final TimeCharacteristic timeCharacteristic = getOperatorConfig().getTimeCharacteristic();
-		final SourceFunction.SourceContext<OUT> ctx;
-		
-		switch (timeCharacteristic) {
-			case EventTime:
-				ctx = new ManualWatermarkContext<>(this, lockingObject, collector);
-				break;
-			case IngestionTime:
-				ctx = new AutomaticWatermarkContext<>(this, lockingObject, collector,
-						getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval());
-				break;
-			case ProcessingTime:
-				ctx = new NonTimestampContext<>(this, lockingObject, collector);
-				break;
-			default:
-				throw new Exception(String.valueOf(timeCharacteristic));
-		}
+		final long watermarkInterval = getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval();
+
+		this.ctx = StreamSourceContexts.getSourceContext(
+			timeCharacteristic, getTimerService(), lockingObject, collector, watermarkInterval);
 
-		// copy to a field to give the 'cancel()' method access
-		this.ctx = ctx;
-		
 		try {
 			userFunction.run(ctx);
 
@@ -122,252 +103,4 @@ public class StreamSource<OUT, SRC extends SourceFunction<OUT>>
 	protected boolean isCanceledOrStopped() {
 		return canceledOrStopped;
 	}
-
-	/**
-	 * Checks whether any asynchronous thread (checkpoint trigger, timer, watermark generator, ...)
-	 * has caused an exception. If one of these threads caused an exception, this method will
-	 * throw that exception.
-	 */
-	void checkAsyncException() {
-		getContainingTask().checkTimerException();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Source contexts for various stream time characteristics
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * A source context that attached {@code -1} as a timestamp to all records, and that
-	 * does not forward watermarks.
-	 */
-	public static class NonTimestampContext<T> implements SourceFunction.SourceContext<T> {
-
-		private final StreamSource<?, ?> owner;
-		private final Object lockingObject;
-		private final Output<StreamRecord<T>> output;
-		private final StreamRecord<T> reuse;
-
-		public NonTimestampContext(StreamSource<?, ?> owner, Object lockingObject, Output<StreamRecord<T>> output) {
-			this.owner = owner;
-			this.lockingObject = lockingObject;
-			this.output = output;
-			this.reuse = new StreamRecord<T>(null);
-		}
-
-		@Override
-		public void collect(T element) {
-			owner.checkAsyncException();
-			synchronized (lockingObject) {
-				output.collect(reuse.replace(element));
-			}
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			// ignore the timestamp
-			collect(element);
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			owner.checkAsyncException();
-			// do nothing else
-		}
-
-		@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 StreamSource<?, ?> owner;
-		private final TimeServiceProvider timeService;
-		private final Object lockingObject;
-		private final Output<StreamRecord<T>> output;
-		private final StreamRecord<T> reuse;
-		
-		private final ScheduledFuture<?> watermarkTimer;
-		private final long watermarkInterval;
-
-		private volatile long nextWatermarkTime;
-
-		public AutomaticWatermarkContext(
-				final StreamSource<?, ?> owner,
-				final Object lockingObjectParam,
-				final Output<StreamRecord<T>> outputParam,
-				final long watermarkInterval) {
-			
-			if (watermarkInterval < 1L) {
-				throw new IllegalArgumentException("The watermark interval cannot be smaller than one.");
-			}
-
-			this.owner = owner;
-			this.timeService = owner.getTimerService();
-			this.lockingObject = lockingObjectParam;
-			this.output = outputParam;
-			this.watermarkInterval = watermarkInterval;
-			this.reuse = new StreamRecord<T>(null);
-
-			long now = this.timeService.getCurrentProcessingTime();
-			this.watermarkTimer = this.timeService.registerTimer(now + watermarkInterval,
-				new WatermarkEmittingTask(this.timeService, lockingObjectParam, outputParam));
-		}
-
-		@Override
-		public void collect(T element) {
-			owner.checkAsyncException();
-			
-			synchronized (lockingObject) {
-				final long currentTime = this.timeService.getCurrentProcessingTime();
-				output.collect(reuse.replace(element, currentTime));
-
-				// this is to avoid lock contention in the lockingObject by
-				// sending the watermark before the firing of the watermark
-				// emission task.
-
-				if (currentTime > nextWatermarkTime) {
-					// in case we jumped some watermarks, recompute the next watermark time
-					final long watermarkTime = currentTime - (currentTime % watermarkInterval);
-					nextWatermarkTime = watermarkTime + watermarkInterval;
-					output.emitWatermark(new Watermark(watermarkTime));
-
-					// we do not need to register another timer here
-					// because the emitting task will do so.
-				}
-			}
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			collect(element);
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			owner.checkAsyncException();
-			
-			if (mark.getTimestamp() == Long.MAX_VALUE) {
-				// allow it since this is the special end-watermark that for example the Kafka source emits
-				synchronized (lockingObject) {
-					nextWatermarkTime = Long.MAX_VALUE;
-					output.emitWatermark(mark);
-				}
-
-				// we can shutdown the timer now, no watermarks will be needed any more
-				watermarkTimer.cancel(true);
-			}
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lockingObject;
-		}
-
-		@Override
-		public void close() {
-			watermarkTimer.cancel(true);
-		}
-
-		private class WatermarkEmittingTask implements Triggerable {
-
-			private final TimeServiceProvider timeService;
-			private final Object lockingObject;
-			private final Output<StreamRecord<T>> output;
-
-			private WatermarkEmittingTask(TimeServiceProvider timeService, Object lock, Output<StreamRecord<T>> output) {
-				this.timeService = timeService;
-				this.lockingObject = lock;
-				this.output = output;
-			}
-
-			@Override
-			public void trigger(long timestamp) {
-				final long currentTime = this.timeService.getCurrentProcessingTime();
-
-				if (currentTime > nextWatermarkTime) {
-					// 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
-					final long watermarkTime = currentTime - (currentTime % watermarkInterval);
-
-					synchronized (lockingObject) {
-						if (currentTime > nextWatermarkTime) {
-							output.emitWatermark(new Watermark(watermarkTime));
-							nextWatermarkTime += watermarkInterval;
-						}
-					}
-				}
-
-				this.timeService.registerTimer(this.timeService.getCurrentProcessingTime() + watermarkInterval,
-					new WatermarkEmittingTask(this.timeService, lockingObject, output));
-			}
-		}
-	}
-
-	/**
-	 * A SourceContext for event time. Sources may directly attach timestamps and generate
-	 * watermarks, but if records are emitted without timestamps, no timestamps are automatically
-	 * generated and attached. The records will simply have no timestamp in that case.
-	 * 
-	 * Streaming topologies can use timestamp assigner functions to override the timestamps
-	 * assigned here.
-	 */
-	public static class ManualWatermarkContext<T> implements SourceFunction.SourceContext<T> {
-
-		private final StreamSource<?, ?> owner;
-		private final Object lockingObject;
-		private final Output<StreamRecord<T>> output;
-		private final StreamRecord<T> reuse;
-
-		public ManualWatermarkContext(StreamSource<?, ?> owner, Object lockingObject, Output<StreamRecord<T>> output) {
-			this.owner = owner;
-			this.lockingObject = lockingObject;
-			this.output = output;
-			this.reuse = new StreamRecord<T>(null);
-		}
-
-		@Override
-		public void collect(T element) {
-			owner.checkAsyncException();
-			
-			synchronized (lockingObject) {
-				output.collect(reuse.replace(element));
-			}
-		}
-
-		@Override
-		public void collectWithTimestamp(T element, long timestamp) {
-			owner.checkAsyncException();
-			
-			synchronized (lockingObject) {
-				output.collect(reuse.replace(element, timestamp));
-			}
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-			owner.checkAsyncException();
-			
-			synchronized (lockingObject) {
-				output.emitWatermark(mark);
-			}
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lockingObject;
-		}
-
-		@Override
-		public void close() {}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java
new file mode 100644
index 0000000..abaf4e7
--- /dev/null
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.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.api.operators;
+
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+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.runtime.tasks.TimeServiceProvider;
+import org.apache.flink.util.Preconditions;
+
+import java.util.concurrent.ScheduledFuture;
+
+/**
+ * Source contexts for various stream time characteristics.
+ */
+public class StreamSourceContexts {
+
+	/**
+	 * Depending on the {@link TimeCharacteristic}, this method will return the adequate
+	 * {@link org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext}. That is:
+	 * <ul>
+	 * <li> {@link TimeCharacteristic#IngestionTime} = {@link AutomaticWatermarkContext}
+	 * <li> {@link TimeCharacteristic#ProcessingTime} = {@link NonTimestampContext}
+	 * <li> {@link TimeCharacteristic#EventTime} = {@link ManualWatermarkContext}
+	 * </ul>
+	 * */
+	public static <OUT> SourceFunction.SourceContext<OUT> getSourceContext(
+			TimeCharacteristic timeCharacteristic, TimeServiceProvider timeService,
+			Object checkpointLock, Output<StreamRecord<OUT>> output, long watermarkInterval) {
+
+		final SourceFunction.SourceContext<OUT> ctx;
+		switch (timeCharacteristic) {
+			case EventTime:
+				ctx = new ManualWatermarkContext<>(checkpointLock, output);
+				break;
+			case IngestionTime:
+				ctx = new AutomaticWatermarkContext<>(timeService, checkpointLock, output, watermarkInterval);
+				break;
+			case ProcessingTime:
+				ctx = new NonTimestampContext<>(checkpointLock, output);
+				break;
+			default:
+				throw new IllegalArgumentException(String.valueOf(timeCharacteristic));
+		}
+		return ctx;
+	}
+
+	/**
+	 * A source context that attached {@code -1} as a timestamp to all records, and that
+	 * does not forward watermarks.
+	 */
+	private static class NonTimestampContext<T> implements SourceFunction.SourceContext<T> {
+
+		private final Object lock;
+		private final Output<StreamRecord<T>> output;
+		private final StreamRecord<T> reuse;
+
+		private NonTimestampContext(Object checkpointLock, Output<StreamRecord<T>> output) {
+			this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpoint lock cannot be null.");
+			this.output = Preconditions.checkNotNull(output, "The output cannot be null.");
+			this.reuse = new StreamRecord<>(null);
+		}
+
+		@Override
+		public void collect(T element) {
+			synchronized (lock) {
+				output.collect(reuse.replace(element));
+			}
+		}
+
+		@Override
+		public void collectWithTimestamp(T element, long timestamp) {
+			// ignore the timestamp
+			collect(element);
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+			// do nothing
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {}
+	}
+
+	/**
+	 * {@link SourceFunction.SourceContext} to be used for sources with automatic timestamps
+	 * and watermark emission.
+	 */
+	private static class AutomaticWatermarkContext<T> implements SourceFunction.SourceContext<T> {
+
+		private final TimeServiceProvider timeService;
+		private final Object lock;
+		private final Output<StreamRecord<T>> output;
+		private final StreamRecord<T> reuse;
+
+		private final ScheduledFuture<?> watermarkTimer;
+		private final long watermarkInterval;
+
+		private volatile long nextWatermarkTime;
+
+		private AutomaticWatermarkContext(
+			final TimeServiceProvider timeService,
+			final Object checkpointLock,
+			final Output<StreamRecord<T>> output,
+			final long watermarkInterval) {
+
+			this.timeService = Preconditions.checkNotNull(timeService, "Time Service cannot be null.");
+			this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpoint lock cannot be null.");
+			this.output = Preconditions.checkNotNull(output, "The output cannot be null.");
+
+			Preconditions.checkArgument(watermarkInterval > 1L, "The watermark interval cannot be smaller than 1 ms.");
+			this.watermarkInterval = watermarkInterval;
+
+			this.reuse = new StreamRecord<>(null);
+
+			long now = this.timeService.getCurrentProcessingTime();
+			this.watermarkTimer = this.timeService.registerTimer(now + watermarkInterval,
+				new WatermarkEmittingTask(this.timeService, lock, output));
+		}
+
+		@Override
+		public void collect(T element) {
+			synchronized (lock) {
+				final long currentTime = this.timeService.getCurrentProcessingTime();
+				output.collect(reuse.replace(element, currentTime));
+
+				// this is to avoid lock contention in the lockingObject by
+				// sending the watermark before the firing of the watermark
+				// emission task.
+
+				if (currentTime > nextWatermarkTime) {
+					// in case we jumped some watermarks, recompute the next watermark time
+					final long watermarkTime = currentTime - (currentTime % watermarkInterval);
+					nextWatermarkTime = watermarkTime + watermarkInterval;
+					output.emitWatermark(new Watermark(watermarkTime));
+
+					// we do not need to register another timer here
+					// because the emitting task will do so.
+				}
+			}
+		}
+
+		@Override
+		public void collectWithTimestamp(T element, long timestamp) {
+			collect(element);
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+
+			if (mark.getTimestamp() == Long.MAX_VALUE) {
+				// allow it since this is the special end-watermark that for example the Kafka source emits
+				synchronized (lock) {
+					nextWatermarkTime = Long.MAX_VALUE;
+					output.emitWatermark(mark);
+				}
+
+				// we can shutdown the timer now, no watermarks will be needed any more
+				if (watermarkTimer != null) {
+					watermarkTimer.cancel(true);
+				}
+			}
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {
+			if (watermarkTimer != null) {
+				watermarkTimer.cancel(true);
+			}
+		}
+
+		private class WatermarkEmittingTask implements Triggerable {
+
+			private final TimeServiceProvider timeService;
+			private final Object lock;
+			private final Output<StreamRecord<T>> output;
+
+			private WatermarkEmittingTask(TimeServiceProvider timeService, Object checkpointLock, Output<StreamRecord<T>> output) {
+				this.timeService = Preconditions.checkNotNull(timeService, "Time Service cannot be null.");
+				this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpoint lock cannot be null.");
+				this.output = Preconditions.checkNotNull(output, "The output cannot be null.");
+			}
+
+			@Override
+			public void trigger(long timestamp) {
+				final long currentTime = timeService.getCurrentProcessingTime();
+
+				if (currentTime > nextWatermarkTime) {
+					// 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
+					final long watermarkTime = currentTime - (currentTime % watermarkInterval);
+
+					synchronized (lock) {
+						if (currentTime > nextWatermarkTime) {
+							output.emitWatermark(new Watermark(watermarkTime));
+							nextWatermarkTime = watermarkTime + watermarkInterval;
+						}
+					}
+				}
+
+				long nextWatermark = currentTime + watermarkInterval;
+				this.timeService.registerTimer(nextWatermark, new WatermarkEmittingTask(this.timeService, lock, output));
+			}
+		}
+	}
+
+	/**
+	 * A SourceContext for event time. Sources may directly attach timestamps and generate
+	 * watermarks, but if records are emitted without timestamps, no timestamps are automatically
+	 * generated and attached. The records will simply have no timestamp in that case.
+	 *
+	 * Streaming topologies can use timestamp assigner functions to override the timestamps
+	 * assigned here.
+	 */
+	private static class ManualWatermarkContext<T> implements SourceFunction.SourceContext<T> {
+
+		private final Object lock;
+		private final Output<StreamRecord<T>> output;
+		private final StreamRecord<T> reuse;
+
+		private ManualWatermarkContext(Object checkpointLock, Output<StreamRecord<T>> output) {
+			this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpoint lock cannot be null.");
+			this.output = Preconditions.checkNotNull(output, "The output cannot be null.");
+			this.reuse = new StreamRecord<>(null);
+		}
+
+		@Override
+		public void collect(T element) {
+			synchronized (lock) {
+				output.collect(reuse.replace(element));
+			}
+		}
+
+		@Override
+		public void collectWithTimestamp(T element, long timestamp) {
+			synchronized (lock) {
+				output.collect(reuse.replace(element, timestamp));
+			}
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+			synchronized (lock) {
+				output.emitWatermark(mark);
+			}
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
index c7ec2ed..4c55055 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncExceptionHandler.java
@@ -18,12 +18,14 @@
 package org.apache.flink.streaming.runtime.tasks;
 
 /**
- * Interface for reporting exceptions that are thrown in (possibly) a different thread.
+ * An interface marking a task as capable of handling exceptions thrown
+ * by different threads, other than the one executing the task itself.
  */
 public interface AsyncExceptionHandler {
 
 	/**
-	 * Registers the given exception.
+	 * Handles an exception thrown by another thread (e.g. a TriggerTask),
+	 * other than the one executing the main task.
 	 */
-	void registerAsyncException(AsynchronousException exception);
+	void handleAsyncException(String message, Throwable exception);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
index ea2b07f..9534b3c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.tasks;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
+import org.apache.flink.util.Preconditions;
 
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
@@ -49,9 +50,9 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 	private DefaultTimeServiceProvider(AsyncExceptionHandler task,
 									ScheduledExecutorService threadPoolExecutor,
 									Object checkpointLock) {
-		this.task = task;
-		this.timerService = threadPoolExecutor;
-		this.checkpointLock = checkpointLock;
+		this.task = Preconditions.checkNotNull(task);
+		this.timerService = Preconditions.checkNotNull(threadPoolExecutor);
+		this.checkpointLock = Preconditions.checkNotNull(checkpointLock);
 	}
 
 	@Override
@@ -99,7 +100,7 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 					target.trigger(timestamp);
 				} catch (Throwable t) {
 					TimerException asyncException = new TimerException(t);
-					exceptionHandler.registerAsyncException(asyncException);
+					exceptionHandler.handleAsyncException("Caught exception while processing timer.", asyncException);
 				}
 			}
 		}
@@ -109,7 +110,7 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider {
 	public static DefaultTimeServiceProvider createForTesting(ScheduledExecutorService executor, Object checkpointLock) {
 		return new DefaultTimeServiceProvider(new AsyncExceptionHandler() {
 			@Override
-			public void registerAsyncException(AsynchronousException exception) {
+			public void handleAsyncException(String message, Throwable exception) {
 				exception.printStackTrace();
 			}
 		}, executor, checkpointLock);

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
index d6d2fb5..cf8853e 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
@@ -64,7 +64,7 @@ public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamO
 		final Object lock = getCheckpointLock();
 		
 		while (running && inputProcessor.processInput(operator, lock)) {
-			checkTimerException();
+
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 9802a16..33317fa 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -53,6 +53,7 @@ import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -158,11 +159,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 
 	private List<Collection<OperatorStateHandle>> lazyRestoreOperatorState;
 
-	/**
-	 * This field is used to forward an exception that is caught in the timer thread or other
-	 * asynchronous Threads. Subclasses must ensure that exceptions stored here get thrown on the
-	 * actual execution Thread. */
-	private volatile AsynchronousException asyncException;
 
 	/** The currently active background materialization threads */
 	private final ClosableRegistry cancelables = new ClosableRegistry();
@@ -301,9 +297,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 			// still let the computation fail
 			tryDisposeAllOperators();
 			disposed = true;
-
-			// Don't forget to check and throw exceptions that happened in async thread one last time
-			checkTimerException();
 		}
 		finally {
 			// clean up everything we initialized
@@ -354,19 +347,6 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 		}
 	}
 
-	/**
-	 * Marks task execution failed for an external reason (a reason other than the task code itself
-	 * throwing an exception). If the task is already in a terminal state
-	 * (such as FINISHED, CANCELED, FAILED), or if the task is already canceling this does nothing.
-	 * Otherwise it sets the state to FAILED, and, if the invokable code is running,
-	 * starts an asynchronous thread that aborts that code.
-	 *
-	 * <p>This method never blocks.</p>
-	 */
-	public void failExternally(Throwable cause) {
-		getEnvironment().failExternally(cause);
-	}
-
 	@Override
 	public final void cancel() throws Exception {
 		isRunning = false;
@@ -898,27 +878,21 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 	}
 
 	/**
-	 * Check whether an exception was thrown in a Thread other than the main Thread. (For example
-	 * in the processing-time trigger Thread). This will rethrow that exception in case on
-	 * occurred.
+	 * Handles an exception thrown by another thread (e.g. a TriggerTask),
+	 * other than the one executing the main task by failing the task entirely.
+	 *
+	 * In more detail, it marks task execution failed for an external reason
+	 * (a reason other than the task code itself throwing an exception). If the task
+	 * is already in a terminal state (such as FINISHED, CANCELED, FAILED), or if the
+	 * task is already canceling this does nothing. Otherwise it sets the state to
+	 * FAILED, and, if the invokable code is running, starts an asynchronous thread
+	 * that aborts that code.
 	 *
-	 * <p>This must be called in the main loop of {@code StreamTask} subclasses to ensure
-	 * that we propagate failures.
+	 * <p>This method never blocks.</p>
 	 */
-	public void checkTimerException() throws AsynchronousException {
-		if (asyncException != null) {
-			throw asyncException;
-		}
-	}
-
 	@Override
-	public void registerAsyncException(AsynchronousException exception) {
-		if (isRunning) {
-			LOG.error("Asynchronous exception registered.", exception);
-		}
-		if (this.asyncException == null) {
-			this.asyncException = exception;
-		}
+	public void handleAsyncException(String message, Throwable exception) {
+		getEnvironment().failExternally(exception);
 	}
 
 	// ------------------------------------------------------------------------
@@ -1030,7 +1004,7 @@ public abstract class StreamTask<OUT, OP extends StreamOperator<OUT>>
 			catch (Exception e) {
 				// registers the exception and tries to fail the whole task
 				AsynchronousException asyncException = new AsynchronousException(e);
-				owner.registerAsyncException(asyncException);
+				owner.handleAsyncException("Failure in asynchronous checkpoint materialization", asyncException);
 			}
 			finally {
 				cancelables.unregisterClosable(this);

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
index 9252063..0197c53 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
@@ -88,7 +88,7 @@ public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputS
 		final Object lock = getCheckpointLock();
 		
 		while (running && inputProcessor.processInput(operator, lock)) {
-			checkTimerException();
+
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
index e8663f5..10b30d0 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorTest.java
@@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.Output;
 import org.apache.flink.streaming.api.operators.StoppableStreamSource;
 import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.operators.StreamSourceContexts;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -187,12 +188,11 @@ public class StreamSourceOperatorTest {
 
 		final List<StreamElement> output = new ArrayList<>();
 
-		StreamSource.AutomaticWatermarkContext<String> ctx =
-			new StreamSource.AutomaticWatermarkContext<>(
-				operator,
-				operator.getContainingTask().getCheckpointLock(),
-				new CollectorOutput<String>(output),
-				operator.getExecutionConfig().getAutoWatermarkInterval());
+		StreamSourceContexts.getSourceContext(TimeCharacteristic.IngestionTime,
+			operator.getContainingTask().getTimerService(),
+			operator.getContainingTask().getCheckpointLock(),
+			new CollectorOutput<String>(output),
+			operator.getExecutionConfig().getAutoWatermarkInterval());
 
 		// periodically emit the watermarks
 		// even though we start from 1 the watermark are still
@@ -218,7 +218,7 @@ public class StreamSourceOperatorTest {
 	private static <T> void setupSourceOperator(StreamSource<T, ?> operator,
 												TimeCharacteristic timeChar,
 												long watermarkInterval,
-												final TimeServiceProvider timeProvider) {
+												final TestTimeServiceProvider timeProvider) {
 
 		ExecutionConfig executionConfig = new ExecutionConfig();
 		executionConfig.setAutoWatermarkInterval(watermarkInterval);
@@ -241,9 +241,6 @@ public class StreamSourceOperatorTest {
 		doAnswer(new Answer<TimeServiceProvider>() {
 			@Override
 			public TimeServiceProvider answer(InvocationOnMock invocation) throws Throwable {
-				if (timeProvider == null) {
-					throw new RuntimeException("The time provider is null.");
-				}
 				return timeProvider;
 			}
 		}).when(mockTask).getTimerService();

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
index 60850d8..0351978 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
@@ -23,7 +23,6 @@ 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.AsyncExceptionHandler;
-import org.apache.flink.streaming.runtime.tasks.AsynchronousException;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
 import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
@@ -39,6 +38,7 @@ import org.powermock.modules.junit4.PowerMockRunner;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.junit.Assert.assertEquals;
 
@@ -52,28 +52,24 @@ public class TimeProviderTest {
 		final OneShotLatch latch = new OneShotLatch();
 
 		final Object lock = new Object();
-		TimeServiceProvider timeServiceProvider = DefaultTimeServiceProvider.create(
-				new AsyncExceptionHandler() {
-					@Override
-					public void registerAsyncException(AsynchronousException exception) {
-						exception.printStackTrace();
-					}
-				},
-				Executors.newSingleThreadScheduledExecutor(),
-				lock);
+		TimeServiceProvider timeServiceProvider = DefaultTimeServiceProvider
+			.createForTesting(Executors.newSingleThreadScheduledExecutor(), lock);
 
 		final List<Long> timestamps = new ArrayList<>();
 
-		long start = System.currentTimeMillis();
 		long interval = 50L;
-
 		final long noOfTimers = 20;
 
 		// we add 2 timers per iteration minus the first that would have a negative timestamp
-		final long expectedNoOfTimers = 2 * noOfTimers - 1;
+		final long expectedNoOfTimers = 2 * noOfTimers;
 
 		for (int i = 0; i < noOfTimers; i++) {
-			double nextTimer = start + i * interval;
+
+			// we add a delay (100ms) so that both timers are inserted before the first is processed.
+			// If not, and given that we add timers out of order, we may have a timer firing
+			// before the next one (with smaller timestamp) is added.
+
+			double nextTimer = timeServiceProvider.getCurrentProcessingTime() + 100 + i * interval;
 
 			timeServiceProvider.registerTimer((long) nextTimer, new Triggerable() {
 				@Override
@@ -88,17 +84,15 @@ public class TimeProviderTest {
 			// add also out-of-order tasks to verify that eventually
 			// they will be executed in the correct order.
 
-			if (i > 0) {
-				timeServiceProvider.registerTimer((long) (nextTimer - 10), new Triggerable() {
-					@Override
-					public void trigger(long timestamp) throws Exception {
-						timestamps.add(timestamp);
-						if (timestamps.size() == expectedNoOfTimers) {
-							latch.trigger();
-						}
+			timeServiceProvider.registerTimer((long) (nextTimer - 10L), new Triggerable() {
+				@Override
+				public void trigger(long timestamp) throws Exception {
+					timestamps.add(timestamp);
+					if (timestamps.size() == expectedNoOfTimers) {
+						latch.trigger();
 					}
-				});
-			}
+				}
+			});
 		}
 
 		if (!latch.isTriggered()) {
@@ -114,15 +108,46 @@ public class TimeProviderTest {
 		long lastTs = Long.MIN_VALUE;
 		for (long timestamp: timestamps) {
 			Assert.assertTrue(timestamp >= lastTs);
+			if (lastTs != Long.MIN_VALUE && counter % 2 == 1) {
+				Assert.assertEquals((timestamp - lastTs), 10);
+			}
 			lastTs = timestamp;
-
-			long expectedTs = start + (counter/2) * interval;
-			Assert.assertEquals(timestamp, (expectedTs + ((counter % 2 == 0) ? 0 : 40)));
 			counter++;
 		}
 	}
 
 	@Test
+	public void testDefaultTimeProviderExceptionHandling() throws InterruptedException {
+		final OneShotLatch latch = new OneShotLatch();
+
+		final AtomicBoolean exceptionWasThrown = new AtomicBoolean(false);
+
+		final Object lock = new Object();
+
+		TimeServiceProvider timeServiceProvider = DefaultTimeServiceProvider
+			.create(new AsyncExceptionHandler() {
+				@Override
+				public void handleAsyncException(String message, Throwable exception) {
+					exceptionWasThrown.compareAndSet(false, true);
+					latch.trigger();
+				}
+			}, Executors.newSingleThreadScheduledExecutor(), lock);
+
+		long now = System.currentTimeMillis();
+		timeServiceProvider.registerTimer(now, new Triggerable() {
+			@Override
+			public void trigger(long timestamp) throws Exception {
+				throw new Exception("Exception in Timer");
+			}
+		});
+
+		if (!latch.isTriggered()) {
+			latch.await();
+		}
+		Assert.assertTrue(exceptionWasThrown.get());
+	}
+
+	@Test
 	public void testTimerSorting() throws Exception {
 
 		final List<Long> result = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
index f33da89..30f38e3 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
@@ -183,13 +183,13 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testWindowTriggerTimeAlignment() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
+		TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
 			Executors.newSingleThreadScheduledExecutor(), lock);
 
 		try {
 			@SuppressWarnings("unchecked")
 			final Output<StreamRecord<String>> mockOut = mock(Output.class);
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			AccumulatingProcessingTimeWindowOperator<String, String, String> op;
 
@@ -201,6 +201,11 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
+			timerService.shutdownService();
+			timerService = DefaultTimeServiceProvider.createForTesting(
+				Executors.newSingleThreadScheduledExecutor(), lock);
+			mockTask = createMockTaskWithTimer(timerService, lock);
+
 			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -209,6 +214,11 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
+			timerService.shutdownService();
+			timerService = DefaultTimeServiceProvider.createForTesting(
+				Executors.newSingleThreadScheduledExecutor(), lock);
+			mockTask = createMockTaskWithTimer(timerService, lock);
+
 			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -217,6 +227,11 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
+			timerService.shutdownService();
+			timerService = DefaultTimeServiceProvider.createForTesting(
+				Executors.newSingleThreadScheduledExecutor(), lock);
+			mockTask = createMockTaskWithTimer(timerService, lock);
+
 			op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -243,7 +258,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 		try {
 			final int windowSize = 50;
 			final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			// tumbling window that triggers every 20 milliseconds
 			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
@@ -297,7 +312,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 
 		try {
 			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 			
 			// tumbling window that triggers every 20 milliseconds
 			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
@@ -359,7 +374,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 
 		try {
 			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			// tumbling window that triggers every 20 milliseconds
 			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
@@ -416,7 +431,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 
 		try {
 			final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			// tumbling window that triggers every 20 milliseconds
 			AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
@@ -743,7 +758,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 
 	// ------------------------------------------------------------------------
 
-	private static StreamTask<?, ?> createMockTask() {
+	private static StreamTask<?, ?> createMockTask(Object lock) {
 		Configuration configuration = new Configuration();
 		configuration.setString(ConfigConstants.STATE_BACKEND, "jobmanager");
 
@@ -751,6 +766,7 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 		when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
 		when(task.getName()).thenReturn("Test task name");
 		when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
+		when(task.getCheckpointLock()).thenReturn(lock);
 
 		final TaskManagerRuntimeInfo mockTaskManagerRuntimeInfo = mock(TaskManagerRuntimeInfo.class);
 		when(mockTaskManagerRuntimeInfo.getConfiguration()).thenReturn(configuration);
@@ -765,9 +781,9 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
 	}
 
 	private static StreamTask<?, ?> createMockTaskWithTimer(
-		final TimeServiceProvider timerService)
+		final TimeServiceProvider timerService, final Object lock)
 	{
-		StreamTask<?, ?> mockTask = createMockTask();
+		StreamTask<?, ?> mockTask = createMockTask(lock);
 		when(mockTask.getTimerService()).thenReturn(timerService);
 		return mockTask;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
index 826b230..7539c2d 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
@@ -191,13 +191,13 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 	@Test
 	public void testWindowTriggerTimeAlignment() throws Exception {
 		final Object lock = new Object();
-		final TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
+		TimeServiceProvider timerService = DefaultTimeServiceProvider.createForTesting(
 			Executors.newSingleThreadScheduledExecutor(), lock);
 
 		try {
 			@SuppressWarnings("unchecked")
 			final Output<StreamRecord<String>> mockOut = mock(Output.class);
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 			
 			AggregatingProcessingTimeWindowOperator<String, String> op;
 
@@ -209,6 +209,11 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
+			timerService.shutdownService();
+			timerService = DefaultTimeServiceProvider.createForTesting(
+				Executors.newSingleThreadScheduledExecutor(), lock);
+			mockTask = createMockTaskWithTimer(timerService, lock);
+
 			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -217,6 +222,11 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
+			timerService.shutdownService();
+			timerService = DefaultTimeServiceProvider.createForTesting(
+				Executors.newSingleThreadScheduledExecutor(), lock);
+			mockTask = createMockTaskWithTimer(timerService, lock);
+
 			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -225,6 +235,11 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			assertTrue(op.getNextEvaluationTime() % 1000 == 0);
 			op.dispose();
 
+			timerService.shutdownService();
+			timerService = DefaultTimeServiceProvider.createForTesting(
+				Executors.newSingleThreadScheduledExecutor(), lock);
+			mockTask = createMockTaskWithTimer(timerService, lock);
+
 			op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
 					StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
 			op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
@@ -257,7 +272,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 							IntSerializer.INSTANCE, tupleSerializer,
 							windowSize, windowSize);
 
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			op.setup(mockTask, createTaskConfig(fieldOneSelector, IntSerializer.INSTANCE, 10), out);
 			op.open();
@@ -309,7 +324,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 			final int windowSize = 50;
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>(windowSize);
 
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
 					new AggregatingProcessingTimeWindowOperator<>(
@@ -377,7 +392,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 		try {
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>(50);
 
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			// tumbling window that triggers every 20 milliseconds
 			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
@@ -448,7 +463,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 
 		try {
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>(50);
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			// tumbling window that triggers every 20 milliseconds
 			AggregatingProcessingTimeWindowOperator<Integer, Tuple2<Integer, Integer>> op =
@@ -508,7 +523,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 
 		try {
 			final CollectingOutput<Tuple2<Integer, Integer>> out = new CollectingOutput<>();
-			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService);
+			final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
 
 			ReduceFunction<Tuple2<Integer, Integer>> failingFunction = new FailingFunction(100);
 
@@ -929,7 +944,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 
 	// ------------------------------------------------------------------------
 	
-	private static StreamTask<?, ?> createMockTask() {
+	private static StreamTask<?, ?> createMockTask(Object lock) {
 		Configuration configuration = new Configuration();
 		configuration.setString(ConfigConstants.STATE_BACKEND, "jobmanager");
 
@@ -937,6 +952,7 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 		when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
 		when(task.getName()).thenReturn("Test task name");
 		when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
+		when(task.getCheckpointLock()).thenReturn(lock);
 
 		final TaskManagerRuntimeInfo mockTaskManagerRuntimeInfo = mock(TaskManagerRuntimeInfo.class);
 		when(mockTaskManagerRuntimeInfo.getConfiguration()).thenReturn(configuration);
@@ -947,9 +963,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
 		return task;
 	}
 
-	private static StreamTask<?, ?> createMockTaskWithTimer(final TimeServiceProvider timerService)
+	private static StreamTask<?, ?> createMockTaskWithTimer(final TimeServiceProvider timerService, final Object lock)
 	{
-		StreamTask<?, ?> mockTask = createMockTask();
+		StreamTask<?, ?> mockTask = createMockTask(lock);
 		when(mockTask.getTimerService()).thenReturn(timerService);
 		return mockTask;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
index f638ddd..9b773d8 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
@@ -102,6 +102,8 @@ public class StreamMockEnvironment implements Environment {
 
 	private final ExecutionConfig executionConfig;
 
+	private volatile boolean wasFailedExternally = false;
+
 	public StreamMockEnvironment(Configuration jobConfig, Configuration taskConfig, ExecutionConfig executionConfig,
 									long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) {
 		this.taskInfo = new TaskInfo("", 1, 0, 1, 0);
@@ -325,7 +327,11 @@ public class StreamMockEnvironment implements Environment {
 
 	@Override
 	public void failExternally(Throwable cause) {
-		throw new UnsupportedOperationException("StreamMockEnvironment does not support external task failure.");
+		this.wasFailedExternally = true;
+	}
+
+	public boolean wasFailedExternally() {
+		return wasFailedExternally;
 	}
 
 	@Override


[17/17] flink git commit: [hotfix] [tests] Remove leftover sysout logging code

Posted by se...@apache.org.
[hotfix] [tests] Remove leftover sysout logging code


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

Branch: refs/heads/master
Commit: 417c5a4b429eb8766ff582d424a64a1cd6aa54d3
Parents: 4fc54e3
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Oct 5 20:10:13 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 20:31:56 2016 +0200

----------------------------------------------------------------------
 .../streaming/runtime/operators/windowing/WindowOperatorTest.java | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/417c5a4b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
index cda6524..ba803e3 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
@@ -837,9 +837,6 @@ public class WindowOperatorTest extends TestLogger {
 		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 4), Long.MAX_VALUE));
 		expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), Long.MAX_VALUE));
 
-		System.out.println("BEFORE GOT: " + outputBeforeClose);
-		System.out.println("GOT: " + testHarness.getOutput());
-
 		TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, Iterables.concat(outputBeforeClose, testHarness.getOutput()), new Tuple2ResultSortComparator());
 
 		testHarness.close();


[10/17] flink git commit: [hotfix] [streaming api] Cleanup watermark initialization in window operator

Posted by se...@apache.org.
[hotfix] [streaming api] Cleanup watermark initialization in window operator


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

Branch: refs/heads/master
Commit: 9d24d51f1d19805e23cab370db0f4bbf1c0038bc
Parents: dd3416f
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Oct 4 23:13:53 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:14 2016 +0200

----------------------------------------------------------------------
 .../runtime/operators/windowing/WindowOperator.java       | 10 +---------
 1 file changed, 1 insertion(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9d24d51f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index ffdf334..f010822 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -63,7 +63,6 @@ import org.apache.flink.streaming.runtime.operators.windowing.functions.Internal
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 
 import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.HashMap;
@@ -158,7 +157,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 	 * To keep track of the current watermark so that we can immediately fire if a trigger
 	 * registers an event time callback for a timestamp that lies in the past.
 	 */
-	protected transient long currentWatermark = Long.MIN_VALUE;
+	protected long currentWatermark = Long.MIN_VALUE;
 
 	protected transient Context context = new Context(null, null);
 
@@ -214,11 +213,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		setChainingStrategy(ChainingStrategy.ALWAYS);
 	}
 
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-		currentWatermark = -1;
-	}
-
 	@Override
 	@SuppressWarnings("unchecked")
 	public final void setInputType(TypeInformation<?> type, ExecutionConfig executionConfig) {
@@ -262,8 +256,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
 		if (windowAssigner instanceof MergingWindowAssigner) {
 			mergingWindowsByKey = new HashMap<>();
 		}
-
-		currentWatermark = Long.MIN_VALUE;
 	}
 
 	@Override


[12/17] flink git commit: [FLINK-4718] [docs] Fix figure about parallel watermarks.

Posted by se...@apache.org.
[FLINK-4718] [docs] Fix figure about parallel watermarks.

This closes #2578


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

Branch: refs/heads/master
Commit: 8000b64ae53e096ef557def1b49fc818b05a6d5f
Parents: 7a76328
Author: Neil Derraugh <ne...@intellifylearning.com>
Authored: Fri Sep 30 09:12:34 2016 -0400
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:14 2016 +0200

----------------------------------------------------------------------
 docs/fig/parallel_streams_watermarks.svg | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/8000b64a/docs/fig/parallel_streams_watermarks.svg
----------------------------------------------------------------------
diff --git a/docs/fig/parallel_streams_watermarks.svg b/docs/fig/parallel_streams_watermarks.svg
index f6a4c4b..d3abde8 100644
--- a/docs/fig/parallel_streams_watermarks.svg
+++ b/docs/fig/parallel_streams_watermarks.svg
@@ -432,7 +432,7 @@ under the License.
          y="195.85332"
          id="text3147"
          xml:space="preserve"
-         style="font-size:7.50095272px;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">B|35</text>
+         style="font-size:7.50095272px;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Verdana">I|35</text>
       <text
          x="195.19138"
          y="151.27718"


[03/17] flink git commit: [hotfix] [core] Minor code cleanup and correction of javadocs for filesystem input stream classes.

Posted by se...@apache.org.
[hotfix] [core] Minor code cleanup and correction of javadocs for filesystem input stream classes.


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

Branch: refs/heads/master
Commit: c62776f0f7ac97f6fd484e2a9e0283074d26a444
Parents: 28b37ef
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Oct 4 13:29:46 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:13 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/core/fs/FSDataInputStream.java |  8 ----
 .../core/fs/local/LocalDataInputStream.java     | 41 +++++++-------------
 .../runtime/fs/hdfs/HadoopDataInputStream.java  | 22 +++++------
 3 files changed, 24 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c62776f0/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java
index c94a71d..6ce1235 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java
@@ -16,13 +16,6 @@
  * limitations under the License.
  */
 
-
-/**
- * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache
- * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
- * additional information regarding copyright ownership. 
- */
-
 package org.apache.flink.core.fs;
 
 import org.apache.flink.annotation.Public;
@@ -32,7 +25,6 @@ import java.io.InputStream;
 
 /**
  * Interface for a data input stream to a file on a {@link FileSystem}.
- * 
  */
 @Public
 public abstract class FSDataInputStream extends InputStream {

http://git-wip-us.apache.org/repos/asf/flink/blob/c62776f0/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java
index 99ca2c4..e7b2828 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.fs.local;
 
 import java.io.File;
@@ -26,36 +25,31 @@ import java.io.IOException;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.core.fs.FSDataInputStream;
 
+import javax.annotation.Nonnull;
+
 /**
  * The <code>LocalDataInputStream</code> class is a wrapper class for a data
  * input stream to the local file system.
- * 
  */
 @Internal
 public class LocalDataInputStream extends FSDataInputStream {
 
-	/**
-	 * The file input stream used to read data.
-	 */
-	private FileInputStream fis = null;
+	/** The file input stream used to read data from.*/
+	private final FileInputStream fis;
 
 	/**
 	 * Constructs a new <code>LocalDataInputStream</code> object from a given {@link File} object.
 	 * 
-	 * @param file
-	 *        the {@link File} object the data stream is written to
-	 * @throws IOException
-	 *         thrown if the data input stream cannot be created
+	 * @param file The File the data stream is read from
+	 * 
+	 * @throws IOException Thrown if the data input stream cannot be created.
 	 */
-	public LocalDataInputStream(final File file) throws IOException {
-
+	public LocalDataInputStream(File file) throws IOException {
 		this.fis = new FileInputStream(file);
 	}
 
-
 	@Override
-	public void seek(final long desired) throws IOException {
-
+	public void seek(long desired) throws IOException {
 		this.fis.getChannel().position(desired);
 	}
 
@@ -64,37 +58,28 @@ public class LocalDataInputStream extends FSDataInputStream {
 		return this.fis.getChannel().position();
 	}
 
-
 	@Override
 	public int read() throws IOException {
-
 		return this.fis.read();
 	}
 
-
 	@Override
-	public int read(final byte[] buffer, final int offset, final int length) throws IOException {
-
+	public int read(@Nonnull byte[] buffer, int offset, int length) throws IOException {
 		return this.fis.read(buffer, offset, length);
 	}
-
-
+	
 	@Override
 	public void close() throws IOException {
-
 		this.fis.close();
 	}
-
-
+	
 	@Override
 	public int available() throws IOException {
 		return this.fis.available();
 	}
-
-
+	
 	@Override
 	public long skip(final long n) throws IOException {
 		return this.fis.skip(n);
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c62776f0/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
index 6eeeb57..8893ba4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
@@ -22,30 +22,30 @@ import java.io.IOException;
 
 import org.apache.flink.core.fs.FSDataInputStream;
 
+import javax.annotation.Nonnull;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
- * Concrete implementation of the {@link FSDataInputStream} for the
- * Hadoop Distributed File System.
+ * Concrete implementation of the {@link FSDataInputStream} for the Hadoop's input streams.
+ * This supports all file systems supported by Hadoop, such as HDFS and S3 (S3a/S3n).
  */
 public final class HadoopDataInputStream extends FSDataInputStream {
 
 	private final org.apache.hadoop.fs.FSDataInputStream fsDataInputStream;
 
 	/**
-	 * Creates a new data input stream from the given HDFS input stream
+	 * Creates a new data input stream from the given Hadoop input stream
 	 * 
-	 * @param fsDataInputStream
-	 *        the HDFS input stream
+	 * @param fsDataInputStream The Hadoop input stream
 	 */
 	public HadoopDataInputStream(org.apache.hadoop.fs.FSDataInputStream fsDataInputStream) {
-		if (fsDataInputStream == null) {
-			throw new NullPointerException();
-		}
-		this.fsDataInputStream = fsDataInputStream;
+		this.fsDataInputStream = checkNotNull(fsDataInputStream);
 	}
 
 
 	@Override
-	public synchronized void seek(long desired) throws IOException {
+	public void seek(long desired) throws IOException {
 		fsDataInputStream.seek(desired);
 	}
 
@@ -65,7 +65,7 @@ public final class HadoopDataInputStream extends FSDataInputStream {
 	}
 
 	@Override
-	public int read(byte[] buffer, int offset, int length) throws IOException {
+	public int read(@Nonnull byte[] buffer, int offset, int length) throws IOException {
 		return fsDataInputStream.read(buffer, offset, length);
 	}
 	


[16/17] flink git commit: [FLINK-4700] [tests] Expand and harden TimeServiceProvider test

Posted by se...@apache.org.
[FLINK-4700] [tests] Expand and harden TimeServiceProvider test


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

Branch: refs/heads/master
Commit: 4fc54e3eb341a049529476ef966380d183d099d4
Parents: 8aea8c8
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Oct 5 16:44:56 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 20:31:56 2016 +0200

----------------------------------------------------------------------
 .../AbstractFetcherTimestampsTest.java          |   2 +-
 .../runtime/operators/TestTimeProviderTest.java | 113 ++++++++
 .../runtime/operators/TimeProviderTest.java     | 269 -------------------
 ...AlignedProcessingTimeWindowOperatorTest.java |   2 +-
 ...AlignedProcessingTimeWindowOperatorTest.java |   2 +-
 .../tasks/DefaultTimeServiceProviderTest.java   | 136 +++++++++-
 6 files changed, 251 insertions(+), 273 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
index c3ba7b7..9b5d2e6 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java
@@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceCont
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
-import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
+import org.apache.flink.streaming.runtime.operators.TestTimeProviderTest.ReferenceSettingExceptionHandler;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java
new file mode 100644
index 0000000..a8f2dc4
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.AsyncExceptionHandler;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
+import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ResultPartitionWriter.class})
+@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"})
+public class TestTimeProviderTest {
+
+	@Test
+	public void testCustomTimeServiceProvider() throws Throwable {
+		TestTimeServiceProvider tp = new TestTimeServiceProvider();
+
+		final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<>();
+		mapTask.setTimeService(tp);
+
+		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 StreamTaskTimerTest.DummyMapFunction<String>());
+		streamConfig.setStreamOperator(mapOperator);
+
+		testHarness.invoke();
+
+		assertEquals(testHarness.getTimerService().getCurrentProcessingTime(), 0);
+
+		tp.setCurrentTime(11);
+		assertEquals(testHarness.getTimerService().getCurrentProcessingTime(), 11);
+
+		tp.setCurrentTime(15);
+		tp.setCurrentTime(16);
+		assertEquals(testHarness.getTimerService().getCurrentProcessingTime(), 16);
+
+		// register 2 tasks
+		mapTask.getTimerService().registerTimer(30, new Triggerable() {
+			@Override
+			public void trigger(long timestamp) {
+
+			}
+		});
+
+		mapTask.getTimerService().registerTimer(40, new Triggerable() {
+			@Override
+			public void trigger(long timestamp) {
+
+			}
+		});
+
+		assertEquals(2, tp.getNumRegisteredTimers());
+
+		tp.setCurrentTime(35);
+		assertEquals(1, tp.getNumRegisteredTimers());
+
+		tp.setCurrentTime(40);
+		assertEquals(0, tp.getNumRegisteredTimers());
+
+		tp.shutdownService();
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static class ReferenceSettingExceptionHandler implements AsyncExceptionHandler {
+
+		private final AtomicReference<Throwable> errorReference;
+
+		public ReferenceSettingExceptionHandler(AtomicReference<Throwable> errorReference) {
+			this.errorReference = errorReference;
+		}
+
+		@Override
+		public void handleAsyncException(String message, Throwable exception) {
+			errorReference.compareAndSet(null, exception);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
deleted file mode 100644
index 8d3e621..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimeProviderTest.java
+++ /dev/null
@@ -1,269 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.typeinfo.BasicTypeInfo;
-import org.apache.flink.core.testutils.OneShotLatch;
-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.AsyncExceptionHandler;
-import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
-import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
-import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness;
-import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
-import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ResultPartitionWriter.class})
-@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"})
-public class TimeProviderTest {
-
-	@Test
-	public void testDefaultTimeProvider() throws InterruptedException {
-		final OneShotLatch latch = new OneShotLatch();
-
-		final Object lock = new Object();
-		final AtomicReference<Throwable> error = new AtomicReference<>();
-		
-		TimeServiceProvider timeServiceProvider = new DefaultTimeServiceProvider(
-				new ReferenceSettingExceptionHandler(error), lock);
-
-		final List<Long> timestamps = new ArrayList<>();
-
-		long interval = 50L;
-		final long noOfTimers = 20;
-
-		// we add 2 timers per iteration minus the first that would have a negative timestamp
-		final long expectedNoOfTimers = 2 * noOfTimers;
-
-		for (int i = 0; i < noOfTimers; i++) {
-
-			// we add a delay (100ms) so that both timers are inserted before the first is processed.
-			// If not, and given that we add timers out of order, we may have a timer firing
-			// before the next one (with smaller timestamp) is added.
-
-			double nextTimer = timeServiceProvider.getCurrentProcessingTime() + 100 + i * interval;
-
-			timeServiceProvider.registerTimer((long) nextTimer, new Triggerable() {
-				@Override
-				public void trigger(long timestamp) throws Exception {
-					timestamps.add(timestamp);
-					if (timestamps.size() == expectedNoOfTimers) {
-						latch.trigger();
-					}
-				}
-			});
-
-			// add also out-of-order tasks to verify that eventually
-			// they will be executed in the correct order.
-
-			timeServiceProvider.registerTimer((long) (nextTimer - 10L), new Triggerable() {
-				@Override
-				public void trigger(long timestamp) throws Exception {
-					timestamps.add(timestamp);
-					if (timestamps.size() == expectedNoOfTimers) {
-						latch.trigger();
-					}
-				}
-			});
-		}
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		Assert.assertEquals(timestamps.size(), expectedNoOfTimers);
-
-		// verify that the tasks are executed
-		// in ascending timestamp order
-
-		int counter = 0;
-		long lastTs = Long.MIN_VALUE;
-		for (long timestamp: timestamps) {
-			Assert.assertTrue(timestamp >= lastTs);
-			if (lastTs != Long.MIN_VALUE && counter % 2 == 1) {
-				Assert.assertEquals((timestamp - lastTs), 10);
-			}
-			lastTs = timestamp;
-			counter++;
-		}
-
-		assertNull(error.get());
-	}
-
-	@Test
-	public void testDefaultTimeProviderExceptionHandling() throws InterruptedException {
-		final OneShotLatch latch = new OneShotLatch();
-
-		final AtomicBoolean exceptionWasThrown = new AtomicBoolean(false);
-
-		final Object lock = new Object();
-
-		TimeServiceProvider timeServiceProvider = new DefaultTimeServiceProvider(
-			new AsyncExceptionHandler() {
-				@Override
-				public void handleAsyncException(String message, Throwable exception) {
-					exceptionWasThrown.compareAndSet(false, true);
-					latch.trigger();
-				}
-			}, lock);
-
-		long now = System.currentTimeMillis();
-		timeServiceProvider.registerTimer(now, new Triggerable() {
-			@Override
-			public void trigger(long timestamp) throws Exception {
-				throw new Exception("Exception in Timer");
-			}
-		});
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-		Assert.assertTrue(exceptionWasThrown.get());
-	}
-
-	@Test
-	public void testTimerSorting() throws Exception {
-
-		final List<Long> result = new ArrayList<>();
-
-		TestTimeServiceProvider provider = new TestTimeServiceProvider();
-
-		provider.registerTimer(45, new Triggerable() {
-			@Override
-			public void trigger(long timestamp) {
-				result.add(timestamp);
-			}
-		});
-
-		provider.registerTimer(50, new Triggerable() {
-			@Override
-			public void trigger(long timestamp) {
-				result.add(timestamp);
-			}
-		});
-
-		provider.registerTimer(30, new Triggerable() {
-			@Override
-			public void trigger(long timestamp) {
-				result.add(timestamp);
-			}
-		});
-
-		provider.registerTimer(50, new Triggerable() {
-			@Override
-			public void trigger(long timestamp) {
-				result.add(timestamp);
-			}
-		});
-
-		Assert.assertEquals(provider.getNumRegisteredTimers(), 4);
-
-		provider.setCurrentTime(100);
-		long seen = 0;
-		for (Long l: result) {
-			Assert.assertTrue(l >= seen);
-			seen = l;
-		}
-	}
-
-	@Test
-	public void testCustomTimeServiceProvider() throws Throwable {
-		TestTimeServiceProvider tp = new TestTimeServiceProvider();
-
-		final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<>();
-		mapTask.setTimeService(tp);
-
-		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 StreamTaskTimerTest.DummyMapFunction<String>());
-		streamConfig.setStreamOperator(mapOperator);
-
-		testHarness.invoke();
-
-		assertEquals(testHarness.getTimerService().getCurrentProcessingTime(), 0);
-
-		tp.setCurrentTime(11);
-		assertEquals(testHarness.getTimerService().getCurrentProcessingTime(), 11);
-
-		tp.setCurrentTime(15);
-		tp.setCurrentTime(16);
-		assertEquals(testHarness.getTimerService().getCurrentProcessingTime(), 16);
-
-		// register 2 tasks
-		mapTask.getTimerService().registerTimer(30, new Triggerable() {
-			@Override
-			public void trigger(long timestamp) {
-
-			}
-		});
-
-		mapTask.getTimerService().registerTimer(40, new Triggerable() {
-			@Override
-			public void trigger(long timestamp) {
-
-			}
-		});
-
-		assertEquals(2, tp.getNumRegisteredTimers());
-
-		tp.setCurrentTime(35);
-		assertEquals(1, tp.getNumRegisteredTimers());
-
-		tp.setCurrentTime(40);
-		assertEquals(0, tp.getNumRegisteredTimers());
-
-		tp.shutdownService();
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static class ReferenceSettingExceptionHandler implements AsyncExceptionHandler {
-
-		private final AtomicReference<Throwable> errorReference;
-
-		public ReferenceSettingExceptionHandler(AtomicReference<Throwable> errorReference) {
-			this.errorReference = errorReference;
-		}
-
-		@Override
-		public void handleAsyncException(String message, Throwable exception) {
-			errorReference.compareAndSet(null, exception);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
index 4c6d391..2f687f6 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
@@ -39,7 +39,7 @@ 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.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
+import org.apache.flink.streaming.runtime.operators.TestTimeProviderTest.ReferenceSettingExceptionHandler;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
index 88e28bc..cd82a9c 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
@@ -40,7 +40,7 @@ import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
+import org.apache.flink.streaming.runtime.operators.TestTimeProviderTest.ReferenceSettingExceptionHandler;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;

http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java
index ae895b6..29e13ed 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProviderTest.java
@@ -19,12 +19,14 @@
 package org.apache.flink.streaming.runtime.tasks;
 
 import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler;
+import org.apache.flink.streaming.runtime.operators.TestTimeProviderTest.ReferenceSettingExceptionHandler;
 import org.apache.flink.streaming.runtime.operators.Triggerable;
 
 import org.junit.Test;
 
+import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -37,6 +39,40 @@ import static org.junit.Assert.fail;
 public class DefaultTimeServiceProviderTest {
 
 	@Test
+	public void testTriggerHoldsLock() throws Exception {
+
+		final Object lock = new Object();
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+
+		final DefaultTimeServiceProvider timer = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(errorRef), lock);
+
+		try {
+			assertEquals(0, timer.getNumTasksScheduled());
+
+			// schedule something
+			ScheduledFuture<?> future = timer.registerTimer(System.currentTimeMillis(), new Triggerable() {
+				@Override
+				public void trigger(long timestamp) {
+					assertTrue(Thread.holdsLock(lock));
+				}
+			});
+
+			// wait until the execution is over
+			future.get();
+			assertEquals(0, timer.getNumTasksScheduled());
+
+			// check that no asynchronous error was reported
+			if (errorRef.get() != null) {
+				throw new Exception(errorRef.get());
+			}
+		}
+		finally {
+			timer.shutdownService();
+		}
+	}
+
+	@Test
 	public void testImmediateShutdown() throws Exception {
 
 		final Object lock = new Object();
@@ -171,6 +207,104 @@ public class DefaultTimeServiceProviderTest {
 			future.cancel(false);
 
 			assertEquals(0, timer.getNumTasksScheduled());
+
+			// check that no asynchronous error was reported
+			if (errorRef.get() != null) {
+				throw new Exception(errorRef.get());
+			}
+		}
+		finally {
+			timer.shutdownService();
+		}
+	}
+
+	@Test
+	public void testExceptionReporting() throws InterruptedException {
+		final AtomicBoolean exceptionWasThrown = new AtomicBoolean(false);
+		final OneShotLatch latch = new OneShotLatch();
+		final Object lock = new Object();
+
+		TimeServiceProvider timeServiceProvider = new DefaultTimeServiceProvider(
+				new AsyncExceptionHandler() {
+					@Override
+					public void handleAsyncException(String message, Throwable exception) {
+						exceptionWasThrown.set(true);
+						latch.trigger();
+					}
+				}, lock);
+		
+		timeServiceProvider.registerTimer(System.currentTimeMillis(), new Triggerable() {
+			@Override
+			public void trigger(long timestamp) throws Exception {
+				throw new Exception("Exception in Timer");
+			}
+		});
+
+		latch.await();
+		assertTrue(exceptionWasThrown.get());
+	}
+
+	@Test
+	public void testTimerSorting() throws Exception {
+		final Object lock = new Object();
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+
+		final DefaultTimeServiceProvider timer = new DefaultTimeServiceProvider(
+				new ReferenceSettingExceptionHandler(errorRef), lock);
+
+		try {
+			final OneShotLatch sync = new OneShotLatch();
+
+			// we block the timer execution to make sure we have all the time
+			// to register some additional timers out of order
+			timer.registerTimer(System.currentTimeMillis(), new Triggerable() {
+				@Override
+				public void trigger(long timestamp) throws Exception {
+					sync.await();
+				}
+			});
+			
+			// schedule two timers out of order something
+			final long now = System.currentTimeMillis();
+			final long time1 = now + 6;
+			final long time2 = now + 5;
+			final long time3 = now + 8;
+			final long time4 = now - 2;
+
+			final ArrayBlockingQueue<Long> timestamps = new ArrayBlockingQueue<>(4);
+			Triggerable trigger = new Triggerable() {
+				@Override
+				public void trigger(long timestamp) {
+					timestamps.add(timestamp);
+				}
+			};
+
+			// schedule
+			ScheduledFuture<?> future1 = timer.registerTimer(time1, trigger);
+			ScheduledFuture<?> future2 = timer.registerTimer(time2, trigger);
+			ScheduledFuture<?> future3 = timer.registerTimer(time3, trigger);
+			ScheduledFuture<?> future4 = timer.registerTimer(time4, trigger);
+
+			// now that everything is scheduled, unblock the timer service
+			sync.trigger();
+
+			// wait until both are complete
+			future1.get();
+			future2.get();
+			future3.get();
+			future4.get();
+
+			// verify that the order is 4 - 2 - 1 - 3
+			assertEquals(4, timestamps.size());
+			assertEquals(time4, timestamps.take().longValue());
+			assertEquals(time2, timestamps.take().longValue());
+			assertEquals(time1, timestamps.take().longValue());
+			assertEquals(time3, timestamps.take().longValue());
+
+			// check that no asynchronous error was reported
+			if (errorRef.get() != null) {
+				throw new Exception(errorRef.get());
+			}
 		}
 		finally {
 			timer.shutdownService();


[08/17] flink git commit: [FLINK-4728] [core, optimizer] Replace reference equality with object equality

Posted by se...@apache.org.
[FLINK-4728] [core,optimizer] Replace reference equality with object equality

Some cases of testing Integer equality using == rather than
Integer.equals(Integer), and some additional cleanup.

This closes #2582


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

Branch: refs/heads/master
Commit: 9206b483b68bb41195bdf2da4f0b9c2de517c031
Parents: 10a42f9
Author: Greg Hogan <co...@greghogan.com>
Authored: Mon Oct 3 13:59:57 2016 -0400
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Oct 5 19:36:13 2016 +0200

----------------------------------------------------------------------
 .../flink/api/common/operators/Ordering.java    |  6 ++---
 .../api/common/operators/util/FieldList.java    |  6 ++---
 .../api/common/io/DelimitedInputFormatTest.java |  1 -
 .../api/common/io/EnumerateNestedFilesTest.java |  1 -
 .../api/common/io/FileInputFormatTest.java      |  2 --
 .../base/OuterJoinOperatorBaseTest.java         | 13 ++++-----
 .../typeutils/base/BigDecComparatorTest.java    |  2 +-
 .../typeutils/base/BigDecSerializerTest.java    |  1 -
 .../typeutils/base/DoubleSerializerTest.java    |  9 +++----
 .../typeutils/base/SqlDateComparatorTest.java   |  2 +-
 .../api/java/typeutils/EitherTypeInfoTest.java  |  2 +-
 .../typeutils/runtime/EitherSerializerTest.java |  2 +-
 .../runtime/TupleComparatorTTT1Test.java        |  5 ++--
 .../apache/flink/types/CopyableValueTest.java   |  6 ++---
 .../java/org/apache/flink/types/RecordTest.java |  2 +-
 .../api/java/functions/FunctionAnnotation.java  | 12 ++++-----
 .../api/java/operators/PartitionOperator.java   |  2 +-
 .../apache/flink/api/java/sca/TaggedValue.java  |  4 +--
 .../java/functions/SemanticPropUtilTest.java    |  7 +++--
 .../api/java/operator/CrossOperatorTest.java    | 16 +++++------
 .../api/java/operator/DistinctOperatorTest.java |  4 +--
 .../flink/api/java/operator/GroupingTest.java   |  4 +--
 .../api/java/operator/JoinOperatorTest.java     | 21 +++++++--------
 .../api/java/operator/MaxByOperatorTest.java    |  2 +-
 .../api/java/operator/MinByOperatorTest.java    |  2 +-
 .../api/java/operator/ReduceOperatorTest.java   |  3 ---
 .../api/java/operator/SortPartitionTest.java    |  4 +--
 .../translation/DistinctTranslationTest.java    |  2 +-
 .../api/java/sca/UdfAnalyzerExamplesTest.java   |  2 +-
 .../flink/api/java/sca/UdfAnalyzerTest.java     |  2 +-
 .../flink/optimizer/dag/BulkIterationNode.java  | 18 ++++++-------
 .../optimizer/dag/WorksetIterationNode.java     |  2 +-
 .../dataproperties/GlobalProperties.java        |  8 +++---
 .../apache/flink/optimizer/plan/PlanNode.java   | 16 +++++------
 .../plandump/PlanJSONDumpGenerator.java         | 28 ++++++++++----------
 .../optimizer/AdditionalOperatorsTest.java      |  2 +-
 .../optimizer/BranchingPlansCompilerTest.java   |  2 +-
 .../optimizer/PartitioningReusageTest.java      |  2 +-
 .../SemanticPropertiesAPIToPlanTest.java        |  2 +-
 .../optimizer/dag/GroupCombineNodeTest.java     |  1 -
 .../GlobalPropertiesMatchingTest.java           | 14 +++++-----
 41 files changed, 116 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
index 7332698..afc659a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.api.common.operators;
 
-import java.util.ArrayList;
-
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.operators.util.FieldList;
 import org.apache.flink.api.common.operators.util.FieldSet;
 
+import java.util.ArrayList;
+
 /**
  * This class represents an ordering on a set of fields. It specifies the fields and order direction
  * (ascending, descending).
@@ -145,7 +145,7 @@ public class Ordering implements Cloneable {
 		}
 		
 		for (int i = 0; i < this.indexes.size(); i++) {
-			if (this.indexes.get(i) != otherOrdering.indexes.get(i)) {
+			if (!this.indexes.get(i).equals(otherOrdering.indexes.get(i))) {
 				return false;
 			}
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/main/java/org/apache/flink/api/common/operators/util/FieldList.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/FieldList.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/FieldList.java
index 15a993c..4cbde56 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/FieldList.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/FieldList.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.api.common.operators.util;
 
+import org.apache.flink.annotation.Internal;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.flink.annotation.Internal;
-
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -166,7 +166,7 @@ public class FieldList extends FieldSet {
 			return false;
 		} else {
 			for (int i = 0; i < this.size(); i++) {
-				if (this.get(i) != list.get(i)) {
+				if (!this.get(i).equals(list.get(i))) {
 					return false;
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
index 8a31099..93d5f9f 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/DelimitedInputFormatTest.java
@@ -38,7 +38,6 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.core.fs.Path;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java
index 1076338..3ac17db 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/EnumerateNestedFilesTest.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.testutils.TestFileUtils;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
index f66bd76..3e5d309 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
@@ -40,8 +40,6 @@ import java.net.URI;
 import java.util.Arrays;
 import java.util.Collections;
 
-import static org.junit.Assert.*;
-
 /**
  * Tests for the FileInputFormat
  */

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java
index 69159f2..683e164 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.api.common.operators.base;
 
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.common.operators.BinaryOperatorInformation;
@@ -29,6 +25,11 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.util.Collector;
 import org.junit.Test;
 
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
 import static org.junit.Assert.assertEquals;
 
 @SuppressWarnings("serial")
@@ -67,7 +68,7 @@ public class OuterJoinOperatorBaseTest implements Serializable {
 
 	@Test
 	public void testFullOuterJoinWithEmptyLeftInput() throws Exception {
-		final List<String> leftInput = Arrays.asList();
+		final List<String> leftInput = Collections.emptyList();
 		final List<String> rightInput = Arrays.asList("foo", "bar", "foobar");
 		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
 		List<String> expected = Arrays.asList("null,bar", "null,foo", "null,foobar");
@@ -77,7 +78,7 @@ public class OuterJoinOperatorBaseTest implements Serializable {
 	@Test
 	public void testFullOuterJoinWithEmptyRightInput() throws Exception {
 		final List<String> leftInput = Arrays.asList("foo", "bar", "foobar");
-		final List<String> rightInput = Arrays.asList();
+		final List<String> rightInput = Collections.emptyList();
 		baseOperator.setOuterJoinType(OuterJoinOperatorBase.OuterJoinType.FULL);
 		List<String> expected = Arrays.asList("bar,null", "foo,null", "foobar,null");
 		testOuterJoin(leftInput, rightInput, expected);

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecComparatorTest.java
index ca5819e..5bfdca2 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecComparatorTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecComparatorTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.api.common.typeutils.base;
 
 import java.math.BigDecimal;
-import java.math.BigInteger;
+
 import org.apache.flink.api.common.typeutils.ComparatorTestBase;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecSerializerTest.java
index fd3cbd5..ff375d1 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecSerializerTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BigDecSerializerTest.java
@@ -19,7 +19,6 @@
 package org.apache.flink.api.common.typeutils.base;
 
 import java.math.BigDecimal;
-import java.math.BigInteger;
 import java.util.Random;
 import org.apache.flink.api.common.typeutils.SerializerTestBase;
 import org.apache.flink.api.common.typeutils.TypeSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/DoubleSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/DoubleSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/DoubleSerializerTest.java
index 543c0e9..2e5de9f 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/DoubleSerializerTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/DoubleSerializerTest.java
@@ -18,11 +18,10 @@
 
 package org.apache.flink.api.common.typeutils.base;
 
-import java.util.Random;
-
 import org.apache.flink.api.common.typeutils.SerializerTestBase;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
+
+import java.util.Random;
 /**
  * A test for the {@link DoubleSerializer}.
  */
@@ -46,8 +45,8 @@ public class DoubleSerializerTest extends SerializerTestBase<Double> {
 	@Override
 	protected Double[] getTestData() {
 		Random rnd = new Random(874597969123412341L);
-		Double rndDouble = rnd.nextDouble() * Double.MAX_VALUE;
-		
+		double rndDouble = rnd.nextDouble() * Double.MAX_VALUE;
+
 		return new Double[] {Double.valueOf(0), Double.valueOf(1), Double.valueOf(-1),
 							Double.valueOf(Double.MAX_VALUE), Double.valueOf(Double.MIN_VALUE),
 							Double.valueOf(rndDouble), Double.valueOf(-rndDouble),

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlDateComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlDateComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlDateComparatorTest.java
index cedefe7..7133b01 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlDateComparatorTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/SqlDateComparatorTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.api.common.typeutils.base;
 
 import java.sql.Date;
-import java.sql.Timestamp;
+
 import org.apache.flink.api.common.typeutils.ComparatorTestBase;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
index 78c10b1..a3d4568 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
@@ -32,7 +32,7 @@ public class EitherTypeInfoTest extends TestLogger {
 
 	Either<Integer, String> intEither = Either.Left(1);
 	Either<Integer, String> stringEither = Either.Right("boo");
-	Either<Integer, Tuple2<Double, Long>> tuple2Either = new Right<>(new Tuple2<Double, Long>(42.0, 2l));
+	Either<Integer, Tuple2<Double, Long>> tuple2Either = new Right<>(new Tuple2<Double, Long>(42.0, 2L));
 
 	@Test
 	public void testEitherTypeEquality() {

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java
index d636d5e..acf0d2e 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java
@@ -60,7 +60,7 @@ public class EitherSerializerTest {
 	public void testEitherWithTuple() {
 
 	Either<Tuple2<Long, Long>, Double>[] testData = new Either[] {
-			Either.Left(new Tuple2<>(2l, 9l)),
+			Either.Left(new Tuple2<>(2L, 9L)),
 			new Left<>(new Tuple2<>(Long.MIN_VALUE, Long.MAX_VALUE)),
 			new Right<>(32.0),
 			Right(Double.MIN_VALUE),

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java
index cf73be2..9db5cd9 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java
@@ -17,14 +17,11 @@
  */
 package org.apache.flink.api.java.typeutils.runtime;
 
-import static org.junit.Assert.assertEquals;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.DoubleComparator;
 import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
-import org.apache.flink.api.common.typeutils.base.IntComparator;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.LongComparator;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.common.typeutils.base.StringComparator;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
@@ -32,6 +29,8 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase;
 
+import static org.junit.Assert.assertEquals;
+
 public class TupleComparatorTTT1Test extends TupleComparatorTestBase<Tuple3<Tuple2<String, Double>, Tuple2<Long, Long>, Tuple2<Integer, Long>>> {
 
 	@SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/types/CopyableValueTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/CopyableValueTest.java b/flink-core/src/test/java/org/apache/flink/types/CopyableValueTest.java
index 76bdece..29dc436 100644
--- a/flink-core/src/test/java/org/apache/flink/types/CopyableValueTest.java
+++ b/flink-core/src/test/java/org/apache/flink/types/CopyableValueTest.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.types;
 
-import static org.junit.Assert.*;
-
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+
 public class CopyableValueTest {
 
 	@Test
@@ -33,7 +33,7 @@ public class CopyableValueTest {
 			new DoubleValue(3.1415926535897932),
 			new FloatValue((float) 3.14159265),
 			new IntValue(42),
-			new LongValue(42l),
+			new LongValue(42L),
 			new NullValue(),
 			new ShortValue((short) 42),
 			new StringValue("QED")

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/test/java/org/apache/flink/types/RecordTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/RecordTest.java b/flink-core/src/test/java/org/apache/flink/types/RecordTest.java
index d7e3edd..a081e8e 100644
--- a/flink-core/src/test/java/org/apache/flink/types/RecordTest.java
+++ b/flink-core/src/test/java/org/apache/flink/types/RecordTest.java
@@ -285,7 +285,7 @@ public class RecordTest {
 			record.setNull(mask);
 	
 			for (int i = 0; i < 58; i++) {
-				if (((1l << i) & mask) != 0) {
+				if (((1L << i) & mask) != 0) {
 					assertTrue(record.getField(i, IntValue.class) == null);
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
index d6fd913..f01d9d8 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/FunctionAnnotation.java
@@ -18,19 +18,19 @@
 
 package org.apache.flink.api.java.functions;
 
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.InvalidProgramException;
+
 import java.lang.annotation.Annotation;
 import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
-import java.lang.annotation.Retention;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.Public;
-import org.apache.flink.api.common.InvalidProgramException;
-
 /**
  * This class defines Java annotations for semantic assertions that can be added to Flink functions.
  * Semantic annotations can help the Flink optimizer to generate more efficient execution plans for Flink programs.

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
index 2ed0300..b3234b8 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java
@@ -106,7 +106,7 @@ public class PartitionOperator<T> extends SingleInputOperator<T, T, PartitionOpe
 
 	/**
 	 * Sets the order of keys for range partitioning.
-	 * NOTE: Only valid for {@link PartitionMethod.RANGE}.
+	 * NOTE: Only valid for {@link PartitionMethod#RANGE}.
 	 *
 	 * @param orders array of orders for each specified partition key
 	 * @return The partitioneOperator with properly set orders for given keys

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/main/java/org/apache/flink/api/java/sca/TaggedValue.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sca/TaggedValue.java b/flink-java/src/main/java/org/apache/flink/api/java/sca/TaggedValue.java
index 43450f7..cf0716d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/sca/TaggedValue.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/sca/TaggedValue.java
@@ -44,7 +44,7 @@ public class TaggedValue extends BasicValue {
 		INPUT_1_ITERABLE, INPUT_2_ITERABLE, INPUT_1_ITERATOR, INPUT_2_ITERATOR, // input iterators
 		ITERATOR_TRUE_ASSUMPTION, // boolean value that is "true" at least once
 		NULL // null
-	};
+	}
 
 	public static enum Input {
 		INPUT_1(0), INPUT_2(1);
@@ -58,7 +58,7 @@ public class TaggedValue extends BasicValue {
 		public int getId() {
 			return id;
 		}
-	};
+	}
 
 	private Tag tag;
 	// only inputs can set this to true

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
index 372c0f7..b845e73 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/functions/SemanticPropUtilTest.java
@@ -19,20 +19,19 @@
 
 package org.apache.flink.api.java.functions;
 
+import org.apache.flink.api.common.operators.DualInputSemanticProperties;
 import org.apache.flink.api.common.operators.SemanticProperties;
 import org.apache.flink.api.common.operators.SemanticProperties.InvalidSemanticAnnotationException;
-import org.apache.flink.api.common.typeutils.CompositeType;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.junit.Assert;
-import org.apache.flink.api.common.operators.DualInputSemanticProperties;
 import org.apache.flink.api.common.operators.SingleInputSemanticProperties;
 import org.apache.flink.api.common.operators.util.FieldSet;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.CompositeType;
 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.typeutils.TupleTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.junit.Test;
 
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
index 474563d..59d2d61 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/CrossOperatorTest.java
@@ -18,18 +18,18 @@
 
 package org.apache.flink.api.java.operator;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Assert;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.tuple.Tuple5;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
 
 public class CrossOperatorTest {
 
@@ -450,7 +450,7 @@ public class CrossOperatorTest {
 		public long myLong;
 		public String myString;
 
-		public CustomType() {};
+		public CustomType() {}
 
 		public CustomType(int i, long l, String s) {
 			myInt = i;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
index 2e9bdf7..0bbeeb2 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/DistinctOperatorTest.java
@@ -214,8 +214,8 @@ public class DistinctOperatorTest {
 		public long myLong;
 		public String myString;
 		
-		public CustomType() {};
-		
+		public CustomType() {}
+
 		public CustomType(int i, long l, String s) {
 			myInt = i;
 			myLong = l;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
index 9220095..18b17b5 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/GroupingTest.java
@@ -606,8 +606,8 @@ public class GroupingTest {
 		public String myString;
 		public Nest nested;
 		
-		public CustomType() {};
-		
+		public CustomType() {}
+
 		public CustomType(int i, long l, String s) {
 			myInt = i;
 			myLong = l;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
index ae23382..0246f60 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/JoinOperatorTest.java
@@ -18,10 +18,6 @@
 
 package org.apache.flink.api.java.operator;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.common.functions.JoinFunction;
 import org.apache.flink.api.common.operators.SemanticProperties;
@@ -32,7 +28,6 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.FunctionAnnotation;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.GroupReduceOperator;
 import org.apache.flink.api.java.operators.JoinOperator;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -44,6 +39,10 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
 import static org.junit.Assert.assertTrue;
 
 @SuppressWarnings("serial")
@@ -1167,7 +1166,7 @@ public class JoinOperatorTest {
 
 		public int myInt;
 
-		public Nested() {};
+		public Nested() {}
 
 		public Nested(int i, long l, String s) {
 			myInt = i;
@@ -1188,7 +1187,7 @@ public class JoinOperatorTest {
 		public String myString;
 		public Nested nest;
 		
-		public NestedCustomType() {};
+		public NestedCustomType() {}
 
 		public NestedCustomType(int i, long l, String s) {
 			myInt = i;
@@ -1214,8 +1213,8 @@ public class JoinOperatorTest {
 		public List<String> countries;
 		public Writable interfaceTest;
 		
-		public CustomType() {};
-		
+		public CustomType() {}
+
 		public CustomType(int i, long l, String s) {
 			myInt = i;
 			myLong = l;
@@ -1242,8 +1241,8 @@ public class JoinOperatorTest {
 		public String myString;
 		public Tuple2<Integer, String> intByString;
 		
-		public CustomTypeWithTuple() {};
-		
+		public CustomTypeWithTuple() {}
+
 		public CustomTypeWithTuple(int i, long l, String s) {
 			myInt = i;
 			myLong = l;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
index 2af8a8c..cbb7690 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/MaxByOperatorTest.java
@@ -211,7 +211,7 @@ public class MaxByOperatorTest {
 		public String myString;
 
 		public CustomType() {
-		};
+		}
 
 		public CustomType(int i, long l, String s) {
 			myInt = i;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
index 5d9c938..b9659c0 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/MinByOperatorTest.java
@@ -212,7 +212,7 @@ public class MinByOperatorTest {
 		public String myString;
 
 		public CustomType() {
-		};
+		}
 
 		public CustomType(int i, long l, String s) {
 			myInt = i;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
index dafc1f2..b7f7555 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/ReduceOperatorTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.api.java.operator;
 
-import org.apache.flink.api.common.functions.GroupReduceFunction;
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.operators.SemanticProperties;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -26,12 +25,10 @@ import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.functions.FunctionAnnotation;
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.GroupReduceOperator;
 import org.apache.flink.api.java.operators.ReduceOperator;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple5;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.util.Collector;
 import org.junit.Test;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
index 3540e6a..c3307ec 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/SortPartitionTest.java
@@ -263,8 +263,8 @@ public class SortPartitionTest {
 		public String myString;
 		public Nest nested;
 		
-		public CustomType() {};
-		
+		public CustomType() {}
+
 		public CustomType(int i, long l, String s) {
 			myInt = i;
 			myLong = l;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
index cbdac4a..27c7b2f 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/translation/DistinctTranslationTest.java
@@ -261,7 +261,7 @@ public class DistinctTranslationTest {
 		private static final long serialVersionUID = 1L;
 		public int myInt;
 
-		public CustomType() {};
+		public CustomType() {}
 
 		public CustomType(int i) {
 			myInt = i;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerExamplesTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerExamplesTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerExamplesTest.java
index 01dc070..72a27b1 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerExamplesTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerExamplesTest.java
@@ -204,7 +204,7 @@ public class UdfAnalyzerExamplesTest {
 				edge = edges.next();
 				Integer otherVertex = edge.getSecondVertex();
 				// collect unique vertices
-				if(!otherVertices.contains(otherVertex) && otherVertex != groupVertex) {
+				if(!otherVertices.contains(otherVertex) && !otherVertex.equals(groupVertex)) {
 					this.otherVertices.add(otherVertex);
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java
index ac35793..c371082 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java
@@ -751,7 +751,7 @@ public class UdfAnalyzerTest {
 		}
 
 		private MyPojo recursiveFunction(MyPojo value) {
-			if (value.field == "xyz") {
+			if (value.field.equals("xyz")) {
 				value.field = value.field + "x";
 				return recursiveFunction(value);
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
index 556e2e3..bf3c4ed 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/BulkIterationNode.java
@@ -18,19 +18,12 @@
 
 package org.apache.flink.optimizer.dag;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
 import org.apache.flink.api.common.ExecutionMode;
 import org.apache.flink.api.common.operators.SemanticProperties;
 import org.apache.flink.api.common.operators.SemanticProperties.EmptySemanticProperties;
 import org.apache.flink.api.common.operators.base.BulkIterationBase;
 import org.apache.flink.optimizer.CompilerException;
 import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.traversals.InterestingPropertyVisitor;
 import org.apache.flink.optimizer.costs.CostEstimator;
 import org.apache.flink.optimizer.dag.WorksetIterationNode.SingleRootJoiner;
 import org.apache.flink.optimizer.dataproperties.GlobalProperties;
@@ -45,12 +38,19 @@ import org.apache.flink.optimizer.plan.BulkPartialSolutionPlanNode;
 import org.apache.flink.optimizer.plan.Channel;
 import org.apache.flink.optimizer.plan.NamedChannel;
 import org.apache.flink.optimizer.plan.PlanNode;
-import org.apache.flink.optimizer.plan.SingleInputPlanNode;
 import org.apache.flink.optimizer.plan.PlanNode.FeedbackPropertiesMeetRequirementsReport;
+import org.apache.flink.optimizer.plan.SingleInputPlanNode;
+import org.apache.flink.optimizer.traversals.InterestingPropertyVisitor;
 import org.apache.flink.optimizer.util.NoOpUnaryUdfOp;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.util.Visitor;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
 /**
  * A node in the optimizer's program representation for a bulk iteration.
  */
@@ -314,7 +314,7 @@ public class BulkIterationNode extends SingleInputNode implements IterationNode
 				
 				FeedbackPropertiesMeetRequirementsReport report = candidate.checkPartialSolutionPropertiesMet(pspn, atEndGlobal, atEndLocal);
 				if (report == FeedbackPropertiesMeetRequirementsReport.NO_PARTIAL_SOLUTION) {
-					; // depends only through broadcast variable on the partial solution
+					// depends only through broadcast variable on the partial solution
 				}
 				else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) {
 					// attach a no-op node through which we create the properties of the original input

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
index 7969a94..d7ccaca 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/WorksetIterationNode.java
@@ -361,7 +361,7 @@ public class WorksetIterationNode extends TwoInputNode implements IterationNode
 																							atEndGlobal, atEndLocal);
 
 				if (report == FeedbackPropertiesMeetRequirementsReport.NO_PARTIAL_SOLUTION) {
-					; // depends only through broadcast variable on the workset solution
+					// depends only through broadcast variable on the workset solution
 				}
 				else if (report == FeedbackPropertiesMeetRequirementsReport.NOT_MET) {
 					// attach a no-op node through which we create the properties of the original input

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java
index e64782f..654b054 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dataproperties/GlobalProperties.java
@@ -18,9 +18,6 @@
 
 package org.apache.flink.optimizer.dataproperties;
 
-import java.util.HashSet;
-import java.util.Set;
-
 import org.apache.flink.api.common.ExecutionMode;
 import org.apache.flink.api.common.distributions.DataDistribution;
 import org.apache.flink.api.common.functions.Partitioner;
@@ -37,6 +34,9 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.HashSet;
+import java.util.Set;
+
 /**
  * This class represents global properties of the data at a certain point in the plan.
  * Global properties are properties that describe data across different partitions, such as
@@ -224,7 +224,7 @@ public class GlobalProperties implements Cloneable {
 			}
 			
 			for (int i = 0; i < this.ordering.getNumberOfFields(); i++) {
-				if (this.ordering.getFieldNumber(i) != o.getFieldNumber(i)) {
+				if (!this.ordering.getFieldNumber(i).equals(o.getFieldNumber(i))) {
 					return false;
 				}
 				

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
index 9505a57..b30fa36 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/PlanNode.java
@@ -18,12 +18,6 @@
 
 package org.apache.flink.optimizer.plan;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.util.FieldSet;
 import org.apache.flink.optimizer.CompilerException;
@@ -39,6 +33,12 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.LocalStrategy;
 import org.apache.flink.util.Visitable;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 /**
  * The representation of a data exchange between to operators. The data exchange can realize a shipping strategy, 
  * which established global properties, and a local strategy, which establishes local properties.
@@ -547,7 +547,7 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 	// --------------------------------------------------------------------------------------------
 	
 	public static enum SourceAndDamReport {
-		NOT_FOUND, FOUND_SOURCE, FOUND_SOURCE_AND_DAM;
+		NOT_FOUND, FOUND_SOURCE, FOUND_SOURCE_AND_DAM
 	}
 	
 	
@@ -568,6 +568,6 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
 		MET,
 		
 		/** Indicates that the question whether the properties are met has been determined false */
-		NOT_MET;
+		NOT_MET
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
index 536e425..16bb47b 100644
--- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
+++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plandump/PlanJSONDumpGenerator.java
@@ -18,19 +18,6 @@
 
 package org.apache.flink.optimizer.plandump;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-
 import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.flink.api.common.operators.CompilerHints;
 import org.apache.flink.optimizer.CompilerException;
@@ -55,6 +42,19 @@ import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.util.StringUtils;
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
 
 public class PlanJSONDumpGenerator {
 	
@@ -160,7 +160,7 @@ public class PlanJSONDumpGenerator {
 			//to set first to false!
 			if (visit(child, writer, first)) {
 				first = false;
-			};
+			}
 		}
 		
 		// check if this node should be skipped from the dump

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
index 656a323..b9b6d8a 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/AdditionalOperatorsTest.java
@@ -77,7 +77,7 @@ public class AdditionalOperatorsTest extends CompilerTestBase {
 		DataSet<Long> set2 = env.generateSequence(0,1);
 
 		set1.crossWithHuge(set2).name("Cross")
-				.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());;
+				.output(new DiscardingOutputFormat<Tuple2<Long, Long>>());
 
 		try {
 			Plan plan = env.createProgramPlan();

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
index 1f8904a..2b3136a 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/BranchingPlansCompilerTest.java
@@ -505,7 +505,7 @@ public class BranchingPlansCompilerTest extends CompilerTestBase {
 
 		loopRes.output(new DiscardingOutputFormat<Long>());
 		loopRes.map(new IdentityMapper<Long>())
-				.output(new DiscardingOutputFormat<Long>());;
+				.output(new DiscardingOutputFormat<Long>());
 
 		Plan plan = env.createProgramPlan();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
index e4cf1c8..ac4c090 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/PartitioningReusageTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
index df13da8..fe79704 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/SemanticPropertiesAPIToPlanTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java
index f4776a0..f3e124b 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dag/GroupCombineNodeTest.java
@@ -21,7 +21,6 @@ package org.apache.flink.optimizer.dag;
 import org.apache.flink.api.common.operators.SemanticProperties;
 import org.apache.flink.api.common.operators.SingleInputSemanticProperties;
 import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase;
-import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
 import org.apache.flink.api.common.operators.util.FieldSet;
 import org.apache.flink.configuration.Configuration;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java
----------------------------------------------------------------------
diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java
index 52826d6..ddfa074 100644
--- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java
+++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/dataproperties/GlobalPropertiesMatchingTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.flink.optimizer.dataproperties;
 
-import static org.junit.Assert.*;
-
 import org.apache.flink.api.common.functions.Partitioner;
 import org.apache.flink.api.common.operators.Order;
 import org.apache.flink.api.common.operators.Ordering;
@@ -28,6 +26,10 @@ import org.apache.flink.api.common.operators.util.FieldSet;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.junit.Test;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class GlobalPropertiesMatchingTest {
 
 	@Test
@@ -180,12 +182,12 @@ public class GlobalPropertiesMatchingTest {
 			assertFalse(req.isMetBy(gp3));
 
 			GlobalProperties gp4 = new GlobalProperties();
-			gp3.setAnyPartitioning(new FieldList(6, 1));
-			assertFalse(req.isMetBy(gp3));
+			gp4.setAnyPartitioning(new FieldList(6, 1));
+			assertFalse(req.isMetBy(gp4));
 
 			GlobalProperties gp5 = new GlobalProperties();
-			gp4.setAnyPartitioning(new FieldList(2));
-			assertFalse(req.isMetBy(gp4));
+			gp5.setAnyPartitioning(new FieldList(2));
+			assertFalse(req.isMetBy(gp5));
 		}
 
 		// match hash partitioning


[05/17] flink git commit: [FLINK-4329] [streaming api] Fix Streaming File Source Timestamps/Watermarks Handling

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
index 247edd6..5275a39 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java
@@ -33,7 +33,7 @@ import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
+import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -92,7 +92,7 @@ public class KeyedOneInputStreamOperatorTestHarness<K, IN, OUT>
 
 	public KeyedOneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator,
 			ExecutionConfig executionConfig,
-			TimeServiceProvider testTimeProvider,
+			TestTimeServiceProvider testTimeProvider,
 			KeySelector<IN, K> keySelector,
 			TypeInformation<K> keyType) {
 		super(operator, executionConfig, testTimeProvider);

http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index d6f46fd..d8a0ee2 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -31,15 +31,16 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
 import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 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.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.AsynchronousException;
 import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider;
 import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -87,6 +88,8 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
 	 */
 	private boolean setupCalled = false;
 
+	private volatile boolean wasFailedExternally = false;
+
 	public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator) {
 		this(operator, new ExecutionConfig());
 	}
@@ -100,7 +103,7 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
 	public OneInputStreamOperatorTestHarness(
 			OneInputStreamOperator<IN, OUT> operator,
 			ExecutionConfig executionConfig,
-			TimeServiceProvider testTimeProvider) {
+			TestTimeServiceProvider testTimeProvider) {
 		this(operator, executionConfig, new Object(), testTimeProvider);
 	}
 
@@ -132,10 +135,10 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
 		doAnswer(new Answer<Void>() {
 			@Override
 			public Void answer(InvocationOnMock invocation) throws Throwable {
-				// do nothing
+				wasFailedExternally = true;
 				return null;
 			}
-		}).when(mockTask).registerAsyncException(any(AsynchronousException.class));
+		}).when(mockTask).handleAsyncException(any(String.class), any(Throwable.class));
 
 		try {
 			doAnswer(new Answer<CheckpointStreamFactory>() {
@@ -161,6 +164,18 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
 		}).when(mockTask).getTimerService();
 	}
 
+	public void setTimeCharacteristic(TimeCharacteristic timeCharacteristic) {
+		this.config.setTimeCharacteristic(timeCharacteristic);
+	}
+
+	public TimeCharacteristic getTimeCharacteristic() {
+		return this.config.getTimeCharacteristic();
+	}
+
+	public boolean wasFailedExternally() {
+		return wasFailedExternally;
+	}
+
 	public void setStateBackend(AbstractStateBackend stateBackend) {
 		this.stateBackend = stateBackend;
 	}