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/04/13 10:30:59 UTC

[01/14] flink git commit: [FLINK-3444] [APIs] Add fromElements method with based class type to avoid the exception.

Repository: flink
Updated Branches:
  refs/heads/master f315c5700 -> db85f3858


[FLINK-3444] [APIs] Add fromElements method with based class type to avoid the exception.

This closes #1857


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

Branch: refs/heads/master
Commit: 6bb085ec6d70e196b7b61bec5f6dc3f924ca7906
Parents: 693d5ab
Author: gallenvara <ga...@126.com>
Authored: Wed Apr 6 16:04:32 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 .../flink/api/java/ExecutionEnvironment.java    | 45 ++++++++++++++++-
 .../flink/api/java/io/FromElementsTest.java     | 51 ++++++++++++++++++++
 .../environment/StreamExecutionEnvironment.java | 33 +++++++++++++
 .../api/StreamExecutionEnvironmentTest.java     | 27 +++++++++++
 4 files changed, 155 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6bb085ec/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
index 1363e26..89c817d 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
@@ -777,7 +777,50 @@ public abstract class ExecutionEnvironment {
 			throw new IllegalArgumentException("The number of elements must not be zero.");
 		}
 		
-		return fromCollection(Arrays.asList(data), TypeExtractor.getForObject(data[0]), Utils.getCallLocationName());
+		TypeInformation<X> typeInfo;
+		try {
+			typeInfo = TypeExtractor.getForObject(data[0]);
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Could not create TypeInformation for type " + data[0].getClass().getName()
+					+ "; please specify the TypeInformation manually via "
+					+ "ExecutionEnvironment#fromElements(Collection, TypeInformation)");
+		}
+
+		return fromCollection(Arrays.asList(data), typeInfo, Utils.getCallLocationName());
+	}
+	
+	/**
+	 * Creates a new data set that contains the given elements. The framework will determine the type according to the 
+	 * based type user supplied. The elements should be the same or be the subclass to the based type. 
+	 * The sequence of elements must not be empty.
+	 * Note that this operation will result in a non-parallel data source, i.e. a data source with
+	 * a parallelism of one.
+	 *
+	 * @param type The base class type for every element in the collection.
+	 * @param data The elements to make up the data set.
+	 * @return A DataSet representing the given list of elements.
+	 */
+	@SafeVarargs
+	public final <X> DataSource<X> fromElements(Class<X> type, X... data) {
+		if (data == null) {
+			throw new IllegalArgumentException("The data must not be null.");
+		}
+		if (data.length == 0) {
+			throw new IllegalArgumentException("The number of elements must not be zero.");
+		}
+		
+		TypeInformation<X> typeInfo;
+		try {
+			typeInfo = TypeExtractor.getForClass(type);
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Could not create TypeInformation for type " + type.getName()
+					+ "; please specify the TypeInformation manually via "
+					+ "ExecutionEnvironment#fromElements(Collection, TypeInformation)");
+		}
+
+		return fromCollection(Arrays.asList(data), typeInfo, Utils.getCallLocationName());
 	}
 	
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/6bb085ec/flink-java/src/test/java/org/apache/flink/api/java/io/FromElementsTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/FromElementsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/FromElementsTest.java
new file mode 100644
index 0000000..2f403aa
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/io/FromElementsTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.io;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.junit.Test;
+
+public class FromElementsTest {
+
+	@Test
+	public void fromElementsWithBaseTypeTest1() {
+		ExecutionEnvironment executionEnvironment = ExecutionEnvironment.getExecutionEnvironment();
+		executionEnvironment.fromElements(ParentType.class, new SubType(1, "Java"), new ParentType(1, "hello"));
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void fromElementsWithBaseTypeTest2() {
+		ExecutionEnvironment executionEnvironment = ExecutionEnvironment.getExecutionEnvironment();
+		executionEnvironment.fromElements(SubType.class, new SubType(1, "Java"), new ParentType(1, "hello"));
+	}
+
+	public static class ParentType {
+		int num;
+		String string;
+		public ParentType(int num, String string) {
+			this.num = num;
+			this.string = string;
+		}
+	}
+
+	public static class SubType extends ParentType{
+		public SubType(int num, String string) {
+			super(num, string);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6bb085ec/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index fb7ec9f..ae4758f 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -673,6 +673,39 @@ public abstract class StreamExecutionEnvironment {
 	}
 
 	/**
+	 * Creates a new data set that contains the given elements. The framework will determine the type according to the 
+	 * based type user supplied. The elements should be the same or be the subclass to the based type. 
+	 * The sequence of elements must not be empty.
+	 * Note that this operation will result in a non-parallel data stream source, i.e. a data stream source with a
+	 * degree of parallelism one.
+	 *
+	 * @param type
+	 * 		The based class type in the collection.
+	 * @param data
+	 * 		The array of elements to create the data stream from.
+	 * @param <OUT>
+	 * 		The type of the returned data stream
+	 * @return The data stream representing the given array of elements
+	 */
+	@SafeVarargs
+	public final <OUT> DataStreamSource<OUT> fromElements(Class<OUT> type, OUT... data) {
+		if (data.length == 0) {
+			throw new IllegalArgumentException("fromElements needs at least one element as argument");
+		}
+
+		TypeInformation<OUT> typeInfo;
+		try {
+			typeInfo = TypeExtractor.getForClass(type);
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Could not create TypeInformation for type " + type.getName()
+					+ "; please specify the TypeInformation manually via "
+					+ "StreamExecutionEnvironment#fromElements(Collection, TypeInformation)");
+		}
+		return fromCollection(Arrays.asList(data), typeInfo);
+	}
+
+	/**
 	 * Creates a data stream from the given non-empty collection. The type of the data stream is that of the
 	 * elements in the collection.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/6bb085ec/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
index 67a4b05..5e596b9 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/StreamExecutionEnvironmentTest.java
@@ -45,6 +45,18 @@ import org.junit.Test;
 public class StreamExecutionEnvironmentTest extends StreamingMultipleProgramsTestBase {
 
 	@Test
+	public void fromElementsWithBaseTypeTest1() {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.fromElements(ParentClass.class, new SubClass(1, "Java"), new ParentClass(1, "hello"));
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void fromElementsWithBaseTypeTest2() {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		env.fromElements(SubClass.class, new SubClass(1, "Java"), new ParentClass(1, "hello"));
+	}
+
+	@Test
 	@SuppressWarnings("unchecked")
 	public void testFromCollectionParallelism() {
 		try {
@@ -159,4 +171,19 @@ public class StreamExecutionEnvironmentTest extends StreamingMultipleProgramsTes
 			throw new UnsupportedOperationException();
 		}
 	}
+
+	public static class ParentClass {
+		int num;
+		String string;
+		public ParentClass(int num, String string) {
+			this.num = num;
+			this.string = string;
+		}
+	}
+
+	public static class SubClass extends ParentClass{
+		public SubClass(int num, String string) {
+			super(num, string);
+		}
+	}
 }


[04/14] flink git commit: [FLINK-3700] [build] Add 'findbugs' (javax.annotation) annotations as a core dependency.

Posted by se...@apache.org.
[FLINK-3700] [build] Add 'findbugs' (javax.annotation) annotations as a core dependency.


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

Branch: refs/heads/master
Commit: 272fd12b41a6e85f0e1825ad4f4593a01f9062aa
Parents: f315c57
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Apr 5 12:37:33 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 pom.xml | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/272fd12b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 82b1567..bdf32d1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -113,6 +113,12 @@ under the License.
 			<version>1.1-SNAPSHOT</version>
 		</dependency>
 
+		<!-- Add the 'javax.annotation' annotations (JSR305), such as '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+		</dependency>
+		
 		<dependency>
 			<groupId>org.apache.commons</groupId>
 			<artifactId>commons-lang3</artifactId>
@@ -191,6 +197,13 @@ under the License.
 		-->
 		<dependencies>
 
+			<!-- This manages the 'javax.annotation' annotations (JSR305) -->
+			<dependency>
+				<groupId>com.google.code.findbugs</groupId>
+				<artifactId>jsr305</artifactId>
+				<version>1.3.9</version>
+			</dependency>
+			
 			<!-- Make sure we use a consistent avro version throughout the project -->
 			<dependency>
 				<groupId>org.apache.avro</groupId>


[14/14] flink git commit: [FLINK-3737] [tests] Adding comment about SOCKS proxy server for WikipediaEditsSourceTest

Posted by se...@apache.org.
[FLINK-3737] [tests] Adding comment about SOCKS proxy server for WikipediaEditsSourceTest

This closes #1872


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

Branch: refs/heads/master
Commit: 342db48b03eceee20d9ec54d97f33054fa0627e8
Parents: b188637
Author: Todd Lisonbee <to...@intel.com>
Authored: Mon Apr 11 00:41:34 2016 -0700
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:55 2016 +0200

----------------------------------------------------------------------
 .../connectors/wikiedits/WikipediaEditsSourceTest.java          | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/342db48b/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java b/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java
index 73ab8bf..0c5d93a 100644
--- a/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java
+++ b/flink-contrib/flink-connector-wikiedits/src/test/java/org/apache/flink/streaming/connectors/wikiedits/WikipediaEditsSourceTest.java
@@ -29,6 +29,11 @@ import static org.junit.Assert.fail;
 
 public class WikipediaEditsSourceTest {
 
+	/**
+	 * NOTE: if you are behind a firewall you may need to use a SOCKS Proxy for this test
+	 *
+	 * @see <a href="http://docs.oracle.com/javase/8/docs/technotes/guides/net/proxies.html">Socks Proxy</a>
+	 */
 	@Test(timeout = 120 * 1000)
 	public void testWikipediaEditsSource() throws Exception {
 


[12/14] flink git commit: [FLINK-3126] [core] Remove accumulator type from "value" in web frontend

Posted by se...@apache.org.
[FLINK-3126] [core] Remove accumulator type from "value" in web frontend

This closes #1868


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

Branch: refs/heads/master
Commit: db85f385846a6541c0707ed1ba6fed78446423b5
Parents: 342db48
Author: Zack Pierce <zp...@newrelic.com>
Authored: Mon Apr 11 10:17:34 2016 -0700
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:55 2016 +0200

----------------------------------------------------------------------
 .../StringifiedAccumulatorResult.java           |  14 +-
 .../StringifiedAccumulatorResultTest.java       | 138 +++++++++++++++++++
 2 files changed, 149 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/db85f385/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java
index a0d1eda..c4faad1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java
@@ -55,6 +55,9 @@ public class StringifiedAccumulatorResult implements java.io.Serializable{
 	//  Utilities
 	// ------------------------------------------------------------------------
 
+	/**
+	 * Flatten a map of accumulator names to Accumulator instances into an array of StringifiedAccumulatorResult values
+     */
 	public static StringifiedAccumulatorResult[] stringifyAccumulatorResults(Map<String, Accumulator<?, ?>> accs) {
 		if (accs == null || accs.isEmpty()) {
 			return new StringifiedAccumulatorResult[0];
@@ -65,9 +68,14 @@ public class StringifiedAccumulatorResult implements java.io.Serializable{
 			int i = 0;
 			for (Map.Entry<String, Accumulator<?, ?>> entry : accs.entrySet()) {
 				StringifiedAccumulatorResult result;
-				Accumulator<?, ?> value = entry.getValue();
-				if (value != null) {
-					result = new StringifiedAccumulatorResult(entry.getKey(), value.getClass().getSimpleName(), value.toString());
+				Accumulator<?, ?> accumulator = entry.getValue();
+				if (accumulator != null) {
+					Object localValue = accumulator.getLocalValue();
+					if (localValue != null) {
+						result = new StringifiedAccumulatorResult(entry.getKey(), accumulator.getClass().getSimpleName(), localValue.toString());
+					} else {
+						result = new StringifiedAccumulatorResult(entry.getKey(), accumulator.getClass().getSimpleName(), "null");
+					}
 				} else {
 					result = new StringifiedAccumulatorResult(entry.getKey(), "null", "null");
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/db85f385/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java
new file mode 100644
index 0000000..e6d637b
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.accumulators;
+
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.api.common.accumulators.SimpleAccumulator;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class StringifiedAccumulatorResultTest {
+
+	@Test
+	public void testSerialization() throws IOException {
+		final String name = "a";
+		final String type = "b";
+		final String value = "c";
+		final StringifiedAccumulatorResult original = new StringifiedAccumulatorResult(name, type, value);
+
+		// Confirm no funny business in the constructor to getter pathway
+		assertEquals(name, original.getName());
+		assertEquals(type, original.getType());
+		assertEquals(value, original.getValue());
+
+		final StringifiedAccumulatorResult copy = CommonTestUtils.createCopySerializable(original);
+
+		// Copy should have equivalent core fields
+		assertEquals(name, copy.getName());
+		assertEquals(type, copy.getType());
+		assertEquals(value, copy.getValue());
+	}
+
+	@Test
+	public void stringifyingResultsShouldIncorporateAccumulatorLocalValueDirectly() {
+		final String name = "a";
+		final int targetValue = 314159;
+		final IntCounter acc = new IntCounter();
+		acc.add(targetValue);
+		final Map<String, Accumulator<?, ?>> accumulatorMap = new HashMap<>();
+		accumulatorMap.put(name, acc);
+
+
+		final StringifiedAccumulatorResult[] results = StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap);
+
+		assertEquals(1, results.length);
+
+		final StringifiedAccumulatorResult firstResult = results[0];
+		assertEquals(name, firstResult.getName());
+		assertEquals("IntCounter", firstResult.getType());
+		assertEquals(Integer.toString(targetValue), firstResult.getValue());
+	}
+
+	@Test
+	public void stringifyingResultsShouldReportNullLocalValueAsNonnullValueString() {
+		final String name = "a";
+		final NullBearingAccumulator acc = new NullBearingAccumulator();
+		final Map<String, Accumulator<?, ?>> accumulatorMap = new HashMap<>();
+		accumulatorMap.put(name, acc);
+
+		final StringifiedAccumulatorResult[] results = StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap);
+
+		assertEquals(1, results.length);
+
+		// Note the use of a String with a content of "null" rather than a null value
+		final StringifiedAccumulatorResult firstResult = results[0];
+		assertEquals(name, firstResult.getName());
+		assertEquals("NullBearingAccumulator", firstResult.getType());
+		assertEquals("null", firstResult.getValue());
+	}
+
+	@Test
+	public void stringifyingResultsShouldReportNullAccumulatorWithNonnullValueAndTypeString() {
+		final String name = "a";
+		final Map<String, Accumulator<?, ?>> accumulatorMap = new HashMap<>();
+		accumulatorMap.put(name, null);
+
+		final StringifiedAccumulatorResult[] results = StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap);
+
+		assertEquals(1, results.length);
+
+		// Note the use of String values with content of "null" rather than null values
+		final StringifiedAccumulatorResult firstResult = results[0];
+		assertEquals(name, firstResult.getName());
+		assertEquals("null", firstResult.getType());
+		assertEquals("null", firstResult.getValue());
+	}
+
+	private static class NullBearingAccumulator implements SimpleAccumulator<Serializable> {
+
+		@Override
+		public void add(Serializable value) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public Serializable getLocalValue() {
+			return null;
+		}
+
+		@Override
+		public void resetLocal() {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public void merge(Accumulator<Serializable, Serializable> other) {
+			throw new UnsupportedOperationException();
+		}
+
+		@Override
+		public Accumulator<Serializable, Serializable> clone() {
+			return new NullBearingAccumulator();
+		}
+	}
+}


[09/14] flink git commit: [FLINK-3375] [kafka connector] Rework/simplify Kafka Connector and have a WatermarkExtractor object per partition

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
new file mode 100644
index 0000000..6bad180
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
@@ -0,0 +1,311 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internal;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.ExceptionProxy;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
+import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.WakeupException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * A fetcher that fetches data from Kafka brokers via the Kafka 0.9 consumer API.
+ * 
+ * @param <T> The type of elements produced by the fetcher.
+ */
+public class Kafka09Fetcher<T> extends AbstractFetcher<T, TopicPartition> implements Runnable {
+
+	private static final Logger LOG = LoggerFactory.getLogger(Kafka09Fetcher.class);
+
+	// ------------------------------------------------------------------------
+
+	/** The schema to convert between Kafka's byte messages, and Flink's objects */
+	private final KeyedDeserializationSchema<T> deserializer;
+
+	/** The subtask's runtime context */
+	private final RuntimeContext runtimeContext;
+
+	/** The configuration for the Kafka consumer */
+	private final Properties kafkaProperties;
+
+	/** The maximum number of milliseconds to wait for a fetch batch */
+	private final long pollTimeout;
+
+	/** Flag whether to register Kafka metrics as Flink accumulators */
+	private final boolean forwardKafkaMetrics;
+
+	/** Mutex to guard against concurrent access to the non-threadsafe Kafka consumer */
+	private final Object consumerLock = new Object();
+
+	/** Reference to the Kafka consumer, once it is created */
+	private volatile KafkaConsumer<byte[], byte[]> consumer;
+
+	/** Reference to the proxy, forwarding exceptions from the fetch thread to the main thread */
+	private volatile ExceptionProxy errorHandler;
+
+	/** Flag to mark the main work loop as alive */
+	private volatile boolean running = true;
+
+	// ------------------------------------------------------------------------
+
+	public Kafka09Fetcher(
+			SourceContext<T> sourceContext,
+			List<KafkaTopicPartition> assignedPartitions,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			StreamingRuntimeContext runtimeContext,
+			KeyedDeserializationSchema<T> deserializer,
+			Properties kafkaProperties,
+			long pollTimeout,
+			boolean forwardKafkaMetrics) throws Exception
+	{
+		super(sourceContext, assignedPartitions, watermarksPeriodic, watermarksPunctuated, runtimeContext);
+
+		this.deserializer = deserializer;
+		this.runtimeContext = runtimeContext;
+		this.kafkaProperties = kafkaProperties;
+		this.pollTimeout = pollTimeout;
+		this.forwardKafkaMetrics = forwardKafkaMetrics;
+
+		// if checkpointing is enabled, we are not automatically committing to Kafka.
+		kafkaProperties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG,
+				Boolean.toString(!runtimeContext.isCheckpointingEnabled()));
+	}
+
+	// ------------------------------------------------------------------------
+	//  Fetcher work methods
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void runFetchLoop() throws Exception {
+		this.errorHandler = new ExceptionProxy(Thread.currentThread());
+
+		// rather than running the main fetch loop directly here, we spawn a dedicated thread
+		// this makes sure that no interrupt() call upon canceling reaches the Kafka consumer code
+		Thread runner = new Thread(this, "Kafka 0.9 Fetcher for " + runtimeContext.getTaskNameWithSubtasks());
+		runner.setDaemon(true);
+		runner.start();
+
+		try {
+			runner.join();
+		} catch (InterruptedException e) {
+			// may be the result of a wake-up after an exception. we ignore this here and only
+			// restore the interruption state
+			Thread.currentThread().interrupt();
+		}
+
+		// make sure we propagate any exception that occurred in the concurrent fetch thread,
+		// before leaving this method
+		this.errorHandler.checkAndThrowException();
+	}
+
+	@Override
+	public void cancel() {
+		// flag the main thread to exit
+		running = false;
+
+		// NOTE:
+		//   - We cannot interrupt the runner thread, because the Kafka consumer may
+		//     deadlock when the thread is interrupted while in certain methods
+		//   - We cannot call close() on the consumer, because it will actually throw
+		//     an exception if a concurrent call is in progress
+
+		// make sure the consumer finds out faster that we are shutting down 
+		if (consumer != null) {
+			consumer.wakeup();
+		}
+	}
+
+	@Override
+	public void run() {
+		// This method initializes the KafkaConsumer and guarantees it is torn down properly.
+		// This is important, because the consumer has multi-threading issues,
+		// including concurrent 'close()' calls.
+
+		final KafkaConsumer<byte[], byte[]> consumer;
+		try {
+			consumer = new KafkaConsumer<>(kafkaProperties);
+		}
+		catch (Throwable t) {
+			running = false;
+			errorHandler.reportError(t);
+			return;
+		}
+
+		// from here on, the consumer will be closed properly
+		try {
+			consumer.assign(convertKafkaPartitions(subscribedPartitions()));
+
+			// register Kafka metrics to Flink accumulators
+			if (forwardKafkaMetrics) {
+				Map<MetricName, ? extends Metric> metrics = consumer.metrics();
+				if (metrics == null) {
+					// MapR's Kafka implementation returns null here.
+					LOG.info("Consumer implementation does not support metrics");
+				} else {
+					// we have metrics, register them where possible
+					for (Map.Entry<MetricName, ? extends Metric> metric : metrics.entrySet()) {
+						String name = "KafkaConsumer-" + metric.getKey().name();
+						DefaultKafkaMetricAccumulator kafkaAccumulator =
+								DefaultKafkaMetricAccumulator.createFor(metric.getValue());
+
+						// best effort: we only add the accumulator if available.
+						if (kafkaAccumulator != null) {
+							runtimeContext.addAccumulator(name, kafkaAccumulator);
+						}
+					}
+				}
+			}
+
+			// seek the consumer to the initial offsets
+			for (KafkaTopicPartitionState<TopicPartition> partition : subscribedPartitions()) {
+				if (partition.isOffsetDefined()) {
+					consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1);
+				}
+			}
+
+			// from now on, external operations may call the consumer
+			this.consumer = consumer;
+
+			// main fetch loop
+			while (running) {
+				// get the next batch of records
+				final ConsumerRecords<byte[], byte[]> records;
+				synchronized (consumerLock) {
+					try {
+						records = consumer.poll(pollTimeout);
+					}
+					catch (WakeupException we) {
+						if (running) {
+							throw we;
+						} else {
+							continue;
+						}
+					}
+				}
+
+				// get the records for each topic partition
+				for (KafkaTopicPartitionState<TopicPartition> partition : subscribedPartitions()) {
+					
+					List<ConsumerRecord<byte[], byte[]>> partitionRecords = records.records(partition.getKafkaPartitionHandle());
+
+					for (ConsumerRecord<byte[], byte[]> record : partitionRecords) {
+						T value = deserializer.deserialize(
+								record.key(), record.value(),
+								record.topic(), record.partition(), record.offset());
+
+						if (deserializer.isEndOfStream(value)) {
+							// end of stream signaled
+							running = false;
+							break;
+						}
+
+						// emit the actual record. this also update offset state atomically
+						// and deals with timestamps and watermark generation
+						emitRecord(value, partition, record.offset());
+					}
+				}
+			}
+			// end main fetch loop
+		}
+		catch (Throwable t) {
+			if (running) {
+				running = false;
+				errorHandler.reportError(t);
+			} else {
+				LOG.debug("Stopped ConsumerThread threw exception", t);
+			}
+		}
+		finally {
+			try {
+				synchronized (consumerLock) {
+					consumer.close();
+				}
+			} catch (Throwable t) {
+				LOG.warn("Error while closing Kafka 0.9 consumer", t);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Kafka 0.9 specific fetcher behavior
+	// ------------------------------------------------------------------------
+
+	@Override
+	public TopicPartition createKafkaPartitionHandle(KafkaTopicPartition partition) {
+		return new TopicPartition(partition.getTopic(), partition.getPartition());
+	}
+
+	@Override
+	public void commitSpecificOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
+		KafkaTopicPartitionState<TopicPartition>[] partitions = subscribedPartitions();
+		Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = new HashMap<>(partitions.length);
+
+		for (KafkaTopicPartitionState<TopicPartition> partition : partitions) {
+			Long offset = offsets.get(partition.getKafkaTopicPartition());
+			if (offset != null) {
+				offsetsToCommit.put(partition.getKafkaPartitionHandle(), new OffsetAndMetadata(offset, ""));
+			}
+		}
+
+		if (this.consumer != null) {
+			synchronized (consumerLock) {
+				this.consumer.commitSync(offsetsToCommit);
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	public static List<TopicPartition> convertKafkaPartitions(KafkaTopicPartitionState<TopicPartition>[] partitions) {
+		ArrayList<TopicPartition> result = new ArrayList<>(partitions.length);
+		for (KafkaTopicPartitionState<TopicPartition> p : partitions) {
+			result.add(p.getKafkaPartitionHandle());
+		}
+		return result;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
index 82e1dce..afb0056 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
@@ -27,11 +27,6 @@ public class Kafka09ITCase extends KafkaConsumerTestBase {
 	// ------------------------------------------------------------------------
 
 	@Test(timeout = 60000)
-	public void testCheckpointing() throws Exception {
-		runCheckpointingTest();
-	}
-
-	@Test(timeout = 60000)
 	public void testFailOnNoBroker() throws Exception {
 		runFailOnNoBrokerTest();
 	}
@@ -41,15 +36,15 @@ public class Kafka09ITCase extends KafkaConsumerTestBase {
 		runSimpleConcurrentProducerConsumerTopology();
 	}
 
-	@Test(timeout = 60000)
-	public void testPunctuatedExplicitWMConsumer() throws Exception {
-		runExplicitPunctuatedWMgeneratingConsumerTest(false);
-	}
+//	@Test(timeout = 60000)
+//	public void testPunctuatedExplicitWMConsumer() throws Exception {
+//		runExplicitPunctuatedWMgeneratingConsumerTest(false);
+//	}
 
-	@Test(timeout = 60000)
-	public void testPunctuatedExplicitWMConsumerWithEmptyTopic() throws Exception {
-		runExplicitPunctuatedWMgeneratingConsumerTest(true);
-	}
+//	@Test(timeout = 60000)
+//	public void testPunctuatedExplicitWMConsumerWithEmptyTopic() throws Exception {
+//		runExplicitPunctuatedWMgeneratingConsumerTest(true);
+//	}
 
 	@Test(timeout = 60000)
 	public void testKeyValueSupport() throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
index a2c4f73..b80a231 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -22,19 +22,23 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.util.TestLogger;
+
 import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.PartitionInfo;
+
 import org.junit.Test;
 import org.junit.runner.RunWith;
+
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
+
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.Properties;
 import java.util.concurrent.Future;
 
@@ -60,7 +64,7 @@ public class KafkaProducerTest extends TestLogger {
 			
 			// partition setup
 			when(kafkaProducerMock.partitionsFor(anyString())).thenReturn(
-					Arrays.asList(new PartitionInfo("mock_topic", 42, null, null, null)));
+					Collections.singletonList(new PartitionInfo("mock_topic", 42, null, null, null)));
 
 			// failure when trying to send an element
 			when(kafkaProducerMock.send(any(ProducerRecord.class), any(Callback.class)))

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention09ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention09ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention09ITCase.java
index 74b35af..c1b21b7 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention09ITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention09ITCase.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.junit.Test;
 
+@SuppressWarnings("serial")
 public class KafkaShortRetention09ITCase extends KafkaShortRetentionTestBase {
 
 	@Test(timeout=60000)

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties
index 6bdfb48..fbeb110 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties
@@ -25,5 +25,6 @@ log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
 
 # suppress the irrelevant (wrong) warnings from the netty channel handler
 log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
-
-
+log4j.logger.org.apache.zookeeper=OFF, testlogger
+log4j.logger.state.change.logger=OFF, testlogger
+log4j.logger.kafka=OFF, testlogger

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
index d9e813f..0ca8fd5 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
@@ -18,427 +18,291 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.commons.collections.map.LinkedMap;
-import org.apache.flink.api.common.ExecutionConfig;
+
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.functions.TimestampAssigner;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaPartitionState;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
-
-import static java.util.Objects.requireNonNull;
-import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.checkArgument;
-
-public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T>
-		implements CheckpointListener, CheckpointedAsynchronously<HashMap<KafkaTopicPartition, Long>>, ResultTypeQueryable<T>, Triggerable {
-
-	// ------------------------------------------------------------------------
 
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class);
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
+/**
+ * Base class of all Flink Kafka Consumer data sources.
+ * This implements the common behavior across all Kafka versions.
+ * 
+ * <p>The Kafka version specific behavior is defined mainly in the specific subclasses of the
+ * {@link AbstractFetcher}.
+ * 
+ * @param <T> The type of records produced by this data source
+ */
+public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T> implements 
+		CheckpointListener,
+		CheckpointedAsynchronously<HashMap<KafkaTopicPartition, Long>>,
+		ResultTypeQueryable<T>
+{
 	private static final long serialVersionUID = -6272159445203409112L;
 
-	/** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid),
-	 * and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */
-	public static final long OFFSET_NOT_SET = -915623761776L;
-
+	protected static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumerBase.class);
+	
 	/** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */
 	public static final int MAX_NUM_PENDING_CHECKPOINTS = 100;
 
-
-	/** The schema to convert between Kafka#s byte messages, and Flink's objects */
+	// ------------------------------------------------------------------------
+	//  configuration state, set on the client relevant for all subtasks
+	// ------------------------------------------------------------------------
+	
+	/** The schema to convert between Kafka's byte messages, and Flink's objects */
 	protected final KeyedDeserializationSchema<T> deserializer;
 
-	// ------  Runtime State  -------
+	/** The set of topic partitions that the source will read */
+	protected List<KafkaTopicPartition> allSubscribedPartitions;
+	
+	/** Optional timestamp extractor / watermark generator that will be run per Kafka partition,
+	 * to exploit per-partition timestamp characteristics.
+	 * The assigner is kept in serialized form, to deserialize it into multiple copies */
+	private SerializedValue<AssignerWithPeriodicWatermarks<T>> periodicWatermarkAssigner;
+	
+	/** Optional timestamp extractor / watermark generator that will be run per Kafka partition,
+	 * to exploit per-partition timestamp characteristics. 
+	 * The assigner is kept in serialized form, to deserialize it into multiple copies */
+	private SerializedValue<AssignerWithPunctuatedWatermarks<T>> punctuatedWatermarkAssigner;
 
+	// ------------------------------------------------------------------------
+	//  runtime state (used individually by each parallel subtask) 
+	// ------------------------------------------------------------------------
+	
 	/** Data for pending but uncommitted checkpoints */
-	protected final LinkedMap pendingCheckpoints = new LinkedMap();
-
-	/**
-	 * Information about the partitions being read by the local consumer. This contains:
-	 * offsets of the last returned elements, and if a timestamp assigner is used, it
-	 * also contains the maximum seen timestamp in the partition and if the partition
-	 * still receives elements or it is inactive.
-	 */
-	protected transient HashMap<KafkaTopicPartition, KafkaPartitionState> partitionState;
+	private final LinkedMap pendingCheckpoints = new LinkedMap();
 
+	/** The fetcher implements the connections to the Kafka brokers */
+	private transient volatile AbstractFetcher<T, ?> kafkaFetcher;
+	
 	/** The offsets to restore to, if the consumer restores state from a checkpoint */
-	protected transient HashMap<KafkaTopicPartition, Long> restoreToOffset;
-
+	private transient volatile HashMap<KafkaTopicPartition, Long> restoreToOffset;
+	
 	/** Flag indicating whether the consumer is still running **/
-	protected volatile boolean running = true;
+	private volatile boolean running = true;
 
 	// ------------------------------------------------------------------------
-	//							WATERMARK EMISSION
-	// ------------------------------------------------------------------------
 
 	/**
-	 * The user-specified methods to extract the timestamps from the records in Kafka, and
-	 * to decide when to emit watermarks.
-	 */
-	private AssignerWithPunctuatedWatermarks<T> punctuatedWatermarkAssigner;
-
-	/**
-	 * The user-specified methods to extract the timestamps from the records in Kafka, and
-	 * to decide when to emit watermarks.
-	 */
-	private AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner;
-
-	private StreamingRuntimeContext runtime = null;
-
-	private SourceContext<T> srcContext = null;
-
-	/**
-	 * The interval between consecutive periodic watermark emissions,
-	 * as configured via the {@link ExecutionConfig#getAutoWatermarkInterval()}.
-	 */
-	private long watermarkInterval = -1;
-
-	/** The last emitted watermark. */
-	private long lastEmittedWatermark = Long.MIN_VALUE;
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler.
-	 *
-	 * <p>To determine which kink of fetcher and offset handler to use, please refer to the docs
-	 * at the beginning of this class.</p>
+	 * Base constructor.
 	 *
 	 * @param deserializer
 	 *           The deserializer to turn raw byte messages into Java/Scala objects.
-	 * @param props
-	 *           The properties that are used to configure both the fetcher and the offset handler.
 	 */
-	public FlinkKafkaConsumerBase(KeyedDeserializationSchema<T> deserializer, Properties props) {
-		this.deserializer = requireNonNull(deserializer, "valueDeserializer");
+	public FlinkKafkaConsumerBase(KeyedDeserializationSchema<T> deserializer) {
+		this.deserializer = checkNotNull(deserializer, "valueDeserializer");
 	}
 
 	/**
-	 * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated manner. Bare in mind
-	 * that the source can either have an {@link AssignerWithPunctuatedWatermarks} or an
-	 * {@link AssignerWithPeriodicWatermarks}, not both.
+	 * This method must be called from the subclasses, to set the list of all subscribed partitions
+	 * that this consumer will fetch from (across all subtasks).
+	 * 
+	 * @param allSubscribedPartitions The list of all partitions that all subtasks together should fetch from.
 	 */
-	public FlinkKafkaConsumerBase<T> setPunctuatedWatermarkEmitter(AssignerWithPunctuatedWatermarks<T> assigner) {
-		checkEmitterDuringInit();
-		this.punctuatedWatermarkAssigner = assigner;
-		return this;
+	protected void setSubscribedPartitions(List<KafkaTopicPartition> allSubscribedPartitions) {
+		checkNotNull(allSubscribedPartitions);
+		this.allSubscribedPartitions = Collections.unmodifiableList(allSubscribedPartitions);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Configuration
+	// ------------------------------------------------------------------------
+	
 	/**
-	 * Specifies an {@link AssignerWithPeriodicWatermarks} to emit watermarks periodically. Bare in mind that the
-	 * source can either have an {@link AssignerWithPunctuatedWatermarks} or an
-	 * {@link AssignerWithPeriodicWatermarks}, not both.
+	 * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated manner.
+	 * The watermark extractor will run per Kafka partition, watermarks will be merged across partitions
+	 * in the same way as in the Flink runtime, when streams are merged.
+	 * 
+	 * <p>When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions,
+	 * the streams from the partitions are unioned in a "first come first serve" fashion. Per-partition
+	 * characteristics are usually lost that way. For example, if the timestamps are strictly ascending
+	 * per Kafka partition, they will not be strictly ascending in the resulting Flink DataStream, if the
+	 * parallel source subtask reads more that one partition.
+	 * 
+	 * <p>Running timestamp extractors / watermark generators directly inside the Kafka source, per Kafka
+	 * partition, allows users to let them exploit the per-partition characteristics.
+	 * 
+	 * <p>Note: One can use either an {@link AssignerWithPunctuatedWatermarks} or an
+	 * {@link AssignerWithPeriodicWatermarks}, not both at the same time.
+	 * 
+	 * @param assigner The timestamp assigner / watermark generator to use.
+	 * @return The consumer object, to allow function chaining.   
 	 */
-	public FlinkKafkaConsumerBase<T> setPeriodicWatermarkEmitter(AssignerWithPeriodicWatermarks<T> assigner) {
-		checkEmitterDuringInit();
-		this.periodicWatermarkAssigner = assigner;
-		return this;
-	}
-
-	/**
-	 * Processes the element after having been read from Kafka and deserialized, and updates the
-	 * last read offset for the specifies partition. These two actions should be performed in
-	 * an atomic way in order to guarantee exactly once semantics.
-	 * @param sourceContext
-	 *           The context the task operates in.
-	 * @param partDescriptor
-	 *            A descriptor containing the topic and the id of the partition.
-	 * @param value
-	 *           The element to process.
-	 * @param offset
-	 *           The offset of the element in the partition.
-	 * */
-	public void processElement(SourceContext<T> sourceContext, KafkaTopicPartition partDescriptor, T value, long offset) {
-		if (punctuatedWatermarkAssigner == null && periodicWatermarkAssigner == null) {
-			// the case where no watermark emitter is specified.
-			sourceContext.collect(value);
-		} else {
-
-			if (srcContext == null) {
-				srcContext = sourceContext;
-			}
-
-			long extractedTimestamp = extractTimestampAndEmitElement(partDescriptor, value);
-
-			// depending on the specified watermark emitter, either send a punctuated watermark,
-			// or set the timer for the first periodic watermark. In the periodic case, we set the timer
-			// only for the first watermark, as it is the trigger() that will set the subsequent ones.
-
-			if (punctuatedWatermarkAssigner != null) {
-				final Watermark nextWatermark = punctuatedWatermarkAssigner
-					.checkAndGetNextWatermark(value, extractedTimestamp);
-				if (nextWatermark != null) {
-					emitWatermarkIfMarkingProgress(sourceContext);
-				}
-			} else if(periodicWatermarkAssigner != null && runtime == null) {
-				runtime = (StreamingRuntimeContext) getRuntimeContext();
-				watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval();
-				if (watermarkInterval > 0) {
-					runtime.registerTimer(System.currentTimeMillis() + watermarkInterval, this);
-				}
-			}
+	public FlinkKafkaConsumerBase<T> setPunctuatedWatermarkEmitter(AssignerWithPunctuatedWatermarks<T> assigner) {
+		checkNotNull(assigner);
+		
+		if (this.periodicWatermarkAssigner != null) {
+			throw new IllegalStateException("A periodic watermark emitter has already been set.");
+		}
+		try {
+			this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner);
+			return this;
+		} catch (Exception e) {
+			throw new IllegalArgumentException("The given assigner is not serializable", e);
 		}
-		updateOffsetForPartition(partDescriptor, offset);
 	}
 
 	/**
-	 * Extract the timestamp from the element based on the user-specified extractor,
-	 * emit the element with the new timestamp, and update the partition monitoring info (if necessary).
-	 * In more detail, upon reception of an element with a timestamp greater than the greatest timestamp
-	 * seen so far in that partition, this method updates the maximum timestamp seen for that partition,
-	 * and marks the partition as {@code active}, i.e. it still receives fresh data.
-	 * @param partDescriptor the partition the new element belongs to.
-	 * @param value the element to be forwarded.
-	 * @return the timestamp of the new element.
+	 * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated manner.
+	 * The watermark extractor will run per Kafka partition, watermarks will be merged across partitions
+	 * in the same way as in the Flink runtime, when streams are merged.
+	 *
+	 * <p>When a subtask of a FlinkKafkaConsumer source reads multiple Kafka partitions,
+	 * the streams from the partitions are unioned in a "first come first serve" fashion. Per-partition
+	 * characteristics are usually lost that way. For example, if the timestamps are strictly ascending
+	 * per Kafka partition, they will not be strictly ascending in the resulting Flink DataStream, if the
+	 * parallel source subtask reads more that one partition.
+	 *
+	 * <p>Running timestamp extractors / watermark generators directly inside the Kafka source, per Kafka
+	 * partition, allows users to let them exploit the per-partition characteristics.
+	 *
+	 * <p>Note: One can use either an {@link AssignerWithPunctuatedWatermarks} or an
+	 * {@link AssignerWithPeriodicWatermarks}, not both at the same time.
+	 *
+	 * @param assigner The timestamp assigner / watermark generator to use.
+	 * @return The consumer object, to allow function chaining.   
 	 */
-	private long extractTimestampAndEmitElement(KafkaTopicPartition partDescriptor, T value) {
-		long extractedTimestamp = getTimestampAssigner().extractTimestamp(value, Long.MIN_VALUE);
-		srcContext.collectWithTimestamp(value, extractedTimestamp);
-		updateMaximumTimestampForPartition(partDescriptor, extractedTimestamp);
-		return extractedTimestamp;
-	}
-
-	/**
-	 * Upon reception of an element with a timestamp greater than the greatest timestamp seen so far in the partition,
-	 * this method updates the maximum timestamp seen for that partition to {@code timestamp}, and marks the partition
-	 * as {@code active}, i.e. it still receives fresh data. If the partition is not known to the system, then a new
-	 * {@link KafkaPartitionState} is created and is associated to the new partition for future monitoring.
-	 * @param partDescriptor
-	 *            A descriptor containing the topic and the id of the partition.
-	 * @param timestamp
-	 *           The timestamp to set the minimum to, if smaller than the already existing one.
-	 * @return {@code true} if the minimum was updated successfully to {@code timestamp}, {@code false}
-	 *           if the previous value is smaller than the provided timestamp
-	 * */
-	private boolean updateMaximumTimestampForPartition(KafkaTopicPartition partDescriptor, long timestamp) {
-		KafkaPartitionState info = getOrInitializeInfo(partDescriptor);
-
-		if(timestamp > info.getMaxTimestamp()) {
-
-			// the flag is set to false as soon as the current partition's max timestamp is sent as a watermark.
-			// if then, and for that partition, only late elements arrive, then the max timestamp will stay the
-			// same, and it will keep the overall system from progressing.
-			// To avoid this, we only mark a partition as active on non-late elements.
-
-			info.setActive(true);
-			info.setMaxTimestamp(timestamp);
-			return  true;
+	public FlinkKafkaConsumerBase<T> setPeriodicWatermarkEmitter(AssignerWithPeriodicWatermarks<T> assigner) {
+		checkNotNull(assigner);
+		
+		if (this.punctuatedWatermarkAssigner != null) {
+			throw new IllegalStateException("A punctuated watermark emitter has already been set.");
+		}
+		try {
+			this.periodicWatermarkAssigner = new SerializedValue<>(assigner);
+			return this;
+		} catch (Exception e) {
+			throw new IllegalArgumentException("The given assigner is not serializable", e);
 		}
-		return false;
 	}
 
-	/**
-	 * Updates the last read offset for the partition specified by the {@code partDescriptor} to {@code offset}.
-	 * If it is the first time we see the partition, then a new {@link KafkaPartitionState} is created to monitor
-	 * this specific partition.
-	 * @param partDescriptor the partition whose info to update.
-	 * @param offset the last read offset of the partition.
-	 */
-	public void updateOffsetForPartition(KafkaTopicPartition partDescriptor, long offset) {
-		KafkaPartitionState info = getOrInitializeInfo(partDescriptor);
-		info.setOffset(offset);
-	}
+	// ------------------------------------------------------------------------
+	//  Work methods
+	// ------------------------------------------------------------------------
 
 	@Override
-	public void trigger(long timestamp) throws Exception {
-		if(this.srcContext == null) {
-			// if the trigger is called before any elements, then we
-			// just set the next timer to fire when it should and we
-			// ignore the triggering as this would produce no results.
-			setNextWatermarkTimer();
-			return;
+	public void run(SourceContext<T> sourceContext) throws Exception {
+		if (allSubscribedPartitions == null) {
+			throw new Exception("The partitions were not set for the consumer");
 		}
-
-		// this is valid because this method is only called when watermarks
-		// are set to be emitted periodically.
-		final Watermark nextWatermark = periodicWatermarkAssigner.getCurrentWatermark();
-		if(nextWatermark != null) {
-			emitWatermarkIfMarkingProgress(srcContext);
-		}
-		setNextWatermarkTimer();
-	}
-
-	/**
-	 * Emits a new watermark, with timestamp equal to the minimum across all the maximum timestamps
-	 * seen per local partition (across all topics). The new watermark is emitted if and only if
-	 * it signals progress in event-time, i.e. if its timestamp is greater than the timestamp of
-	 * the last emitted watermark. In addition, this method marks as inactive the partition whose
-	 * timestamp was emitted as watermark, i.e. the one with the minimum across the maximum timestamps
-	 * of the local partitions. This is done to avoid not making progress because
-	 * a partition stopped receiving data. The partition is going to be marked as {@code active}
-	 * as soon as the <i>next non-late</i> element arrives.
-	 *
-	 * @return {@code true} if the Watermark was successfully emitted, {@code false} otherwise.
-	 */
-	private boolean emitWatermarkIfMarkingProgress(SourceFunction.SourceContext<T> sourceContext) {
-		Tuple2<KafkaTopicPartition, Long> globalMinTs = getMinTimestampAcrossAllTopics();
-		if(globalMinTs.f0 != null ) {
-			synchronized (sourceContext.getCheckpointLock()) {
-				long minTs = globalMinTs.f1;
-				if(minTs > lastEmittedWatermark) {
-					lastEmittedWatermark = minTs;
-					Watermark toEmit = new Watermark(minTs);
-					sourceContext.emitWatermark(toEmit);
-					return true;
-				}
+		
+		// figure out which partitions this subtask should process
+		final List<KafkaTopicPartition> thisSubtaskPartitions = assignPartitions(allSubscribedPartitions,
+				getRuntimeContext().getNumberOfParallelSubtasks(), getRuntimeContext().getIndexOfThisSubtask());
+		
+		// we need only do work, if we actually have partitions assigned
+		if (!thisSubtaskPartitions.isEmpty()) {
+
+			// (1) create the fetcher that will communicate with the Kafka brokers
+			final AbstractFetcher<T, ?> fetcher = createFetcher(
+					sourceContext, thisSubtaskPartitions, 
+					periodicWatermarkAssigner, punctuatedWatermarkAssigner,
+					(StreamingRuntimeContext) getRuntimeContext());
+
+			// (2) set the fetcher to the restored checkpoint offsets
+			if (restoreToOffset != null) {
+				fetcher.restoreOffsets(restoreToOffset);
 			}
-		}
-		return false;
-	}
 
-	/**
-	 * Kafka sources with timestamp extractors are expected to keep the maximum timestamp seen per
-	 * partition they are reading from. This is to mark the per-partition event-time progress.
-	 *
-	 * This method iterates this list, and returns the minimum timestamp across these per-partition
-	 * max timestamps, and across all topics. In addition to this information, it also returns the topic and
-	 * the partition within the topic the timestamp belongs to.
-	 */
-	private Tuple2<KafkaTopicPartition, Long> getMinTimestampAcrossAllTopics() {
-		Tuple2<KafkaTopicPartition, Long> minTimestamp = new Tuple2<>(null, Long.MAX_VALUE);
-		for(Map.Entry<KafkaTopicPartition, KafkaPartitionState> entries: partitionState.entrySet()) {
-			KafkaTopicPartition part = entries.getKey();
-			KafkaPartitionState info = entries.getValue();
-
-			if(partitionIsActive(part) && info.getMaxTimestamp() < minTimestamp.f1) {
-				minTimestamp.f0 = part;
-				minTimestamp.f1 = info.getMaxTimestamp();
+			// publish the reference, for snapshot-, commit-, and cancel calls
+			// IMPORTANT: We can only do that now, because only now will calls to
+			//            the fetchers 'snapshotCurrentState()' method return at least
+			//            the restored offsets
+			this.kafkaFetcher = fetcher;
+			if (!running) {
+				return;
 			}
+			
+			// (3) run the fetcher' main work method
+			fetcher.runFetchLoop();
 		}
-
-		if(minTimestamp.f0 != null) {
-			// it means that we have a winner and we have to set its flag to
-			// inactive, until its next non-late element.
-			KafkaTopicPartition partitionDescriptor = minTimestamp.f0;
-			setActiveFlagForPartition(partitionDescriptor, false);
-		}
-
-		return minTimestamp;
-	}
-
-	/**
-	 * Sets the {@code active} flag for a given partition of a topic to {@code isActive}.
-	 * This flag signals if the partition is still receiving data and it is used to avoid the case
-	 * where a partition stops receiving data, so its max seen timestamp does not advance, and it
-	 * holds back the progress of the watermark for all partitions. Note that if the partition is
-	 * not known to the system, then a new {@link KafkaPartitionState} is created and is associated
-	 * to the new partition for future monitoring.
-	 *
-	 * @param partDescriptor
-	 * 				A descriptor containing the topic and the id of the partition.
-	 * @param isActive
-	 * 				The value {@code true} or {@code false} to set the flag to.
-	 */
-	private void setActiveFlagForPartition(KafkaTopicPartition partDescriptor, boolean isActive) {
-		KafkaPartitionState info = getOrInitializeInfo(partDescriptor);
-		info.setActive(isActive);
-	}
-
-	/**
-	 * Gets the statistics for a given partition specified by the {@code partition} argument.
-	 * If it is the first time we see this partition, a new {@link KafkaPartitionState} data structure
-	 * is initialized to monitor it from now on. This method never throws a {@link NullPointerException}.
-	 * @param partition the partition to be fetched.
-	 * @return the gathered statistics for that partition.
-	 * */
-	private KafkaPartitionState getOrInitializeInfo(KafkaTopicPartition partition) {
-		KafkaPartitionState info = partitionState.get(partition);
-		if(info == null) {
-			info = new KafkaPartitionState(partition.getPartition(), FlinkKafkaConsumerBase.OFFSET_NOT_SET);
-			partitionState.put(partition, info);
+		else {
+			// this source never completes, so emit a Long.MAX_VALUE watermark
+			// to not block watermark forwarding
+			sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
+
+			// wait until this is canceled
+			final Object waitLock = new Object();
+			while (running) {
+				try {
+					//noinspection SynchronizationOnLocalVariableOrMethodParameter
+					synchronized (waitLock) {
+						waitLock.wait();
+					}
+				}
+				catch (InterruptedException e) {
+					if (!running) {
+						// restore the interrupted state, and fall through the loop
+						Thread.currentThread().interrupt();
+					}
+				}
+			}
 		}
-		return info;
 	}
 
-	/**
-	 * Checks if a partition of a topic is still active, i.e. if it still receives data.
-	 * @param partDescriptor
-	 *          A descriptor containing the topic and the id of the partition.
-	 * */
-	private boolean partitionIsActive(KafkaTopicPartition partDescriptor) {
-		KafkaPartitionState info = partitionState.get(partDescriptor);
-		if(info == null) {
-			throw new RuntimeException("Unknown Partition: Topic=" + partDescriptor.getTopic() +
-				" Partition=" + partDescriptor.getPartition());
+	@Override
+	public void cancel() {
+		// set ourselves as not running
+		running = false;
+		
+		// abort the fetcher, if there is one
+		if (kafkaFetcher != null) {
+			kafkaFetcher.cancel();
 		}
-		return info.isActive();
-	}
 
-	private TimestampAssigner<T> getTimestampAssigner() {
-		checkEmitterStateAfterInit();
-		return periodicWatermarkAssigner != null ? periodicWatermarkAssigner : punctuatedWatermarkAssigner;
-	}
-
-	private void setNextWatermarkTimer() {
-		long timeToNextWatermark = System.currentTimeMillis() + watermarkInterval;
-		runtime.registerTimer(timeToNextWatermark, this);
-	}
-
-	private void checkEmitterDuringInit() {
-		if(periodicWatermarkAssigner != null) {
-			throw new RuntimeException("A periodic watermark emitter has already been provided.");
-		} else if(punctuatedWatermarkAssigner != null) {
-			throw new RuntimeException("A punctuated watermark emitter has already been provided.");
-		}
+		// there will be an interrupt() call to the main thread anyways
 	}
 
-	private void checkEmitterStateAfterInit() {
-		if(periodicWatermarkAssigner == null && punctuatedWatermarkAssigner == null) {
-			throw new RuntimeException("The timestamp assigner has not been initialized.");
-		} else if(periodicWatermarkAssigner != null && punctuatedWatermarkAssigner != null) {
-			throw new RuntimeException("The source can either have an assigner with punctuated " +
-				"watermarks or one with periodic watermarks, not both.");
+	@Override
+	public void close() throws Exception {
+		// pretty much the same logic as cancelling
+		try {
+			cancel();
+		} finally {
+			super.close();
 		}
 	}
-
+	
 	// ------------------------------------------------------------------------
 	//  Checkpoint and restore
 	// ------------------------------------------------------------------------
-
-	HashMap<KafkaTopicPartition, KafkaPartitionState> restoreInfoFromCheckpoint() {
-		HashMap<KafkaTopicPartition, KafkaPartitionState> partInfo = new HashMap<>(restoreToOffset.size());
-		for(Map.Entry<KafkaTopicPartition, Long> offsets: restoreToOffset.entrySet()) {
-			KafkaTopicPartition key = offsets.getKey();
-			partInfo.put(key, new KafkaPartitionState(key.getPartition(), offsets.getValue()));
-		}
-		return partInfo;
-	}
-
+	
 	@Override
 	public HashMap<KafkaTopicPartition, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		if (partitionState == null) {
-			LOG.debug("snapshotState() requested on not yet opened source; returning null.");
-			return null;
-		}
 		if (!running) {
 			LOG.debug("snapshotState() called on closed source");
 			return null;
 		}
-
-		HashMap<KafkaTopicPartition, Long> currentOffsets = new HashMap<>();
-		for (Map.Entry<KafkaTopicPartition, KafkaPartitionState> entry: partitionState.entrySet()) {
-			currentOffsets.put(entry.getKey(), entry.getValue().getOffset());
+		
+		final AbstractFetcher<?, ?> fetcher = this.kafkaFetcher;
+		if (fetcher == null) {
+			// the fetcher has not yet been initialized, which means we need to return the
+			// originally restored offsets
+			return restoreToOffset;
 		}
 
+		HashMap<KafkaTopicPartition, Long> currentOffsets = fetcher.snapshotCurrentState();
+
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}",
 					KafkaTopicPartition.toString(currentOffsets), checkpointId, checkpointTimestamp);
@@ -447,7 +311,8 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 		// the map cannot be asynchronously updated, because only one checkpoint call can happen
 		// on this function at a time: either snapshotState() or notifyCheckpointComplete()
 		pendingCheckpoints.put(checkpointId, currentOffsets);
-			
+		
+		// truncate the map, to prevent infinite growth
 		while (pendingCheckpoints.size() > MAX_NUM_PENDING_CHECKPOINTS) {
 			pendingCheckpoints.remove(0);
 		}
@@ -457,51 +322,49 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 
 	@Override
 	public void restoreState(HashMap<KafkaTopicPartition, Long> restoredOffsets) {
-		LOG.info("Setting restore state in Kafka");
+		LOG.info("Setting restore state in the FlinkKafkaConsumer");
 		restoreToOffset = restoredOffsets;
 	}
 
 	@Override
 	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		if (partitionState == null) {
-			LOG.debug("notifyCheckpointComplete() called on uninitialized source");
-			return;
-		}
 		if (!running) {
 			LOG.debug("notifyCheckpointComplete() called on closed source");
 			return;
 		}
+
+		final AbstractFetcher<?, ?> fetcher = this.kafkaFetcher;
+		if (fetcher == null) {
+			LOG.debug("notifyCheckpointComplete() called on uninitialized source");
+			return;
+		}
 		
 		// only one commit operation must be in progress
 		if (LOG.isDebugEnabled()) {
-			LOG.debug("Committing offsets externally for checkpoint {}", checkpointId);
+			LOG.debug("Committing offsets to Kafka/ZooKeeper for checkpoint " + checkpointId);
 		}
 
 		try {
-			HashMap<KafkaTopicPartition, Long> checkpointOffsets;
-	
-			// the map may be asynchronously updates when snapshotting state, so we synchronize
-			synchronized (pendingCheckpoints) {
-				final int posInMap = pendingCheckpoints.indexOf(checkpointId);
-				if (posInMap == -1) {
-					LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
-					return;
-				}
+			final int posInMap = pendingCheckpoints.indexOf(checkpointId);
+			if (posInMap == -1) {
+				LOG.warn("Received confirmation for unknown checkpoint id {}", checkpointId);
+				return;
+			}
 
-				//noinspection unchecked
-				checkpointOffsets = (HashMap<KafkaTopicPartition, Long>) pendingCheckpoints.remove(posInMap);
+			@SuppressWarnings("unchecked")
+			HashMap<KafkaTopicPartition, Long> checkpointOffsets = 
+					(HashMap<KafkaTopicPartition, Long>) pendingCheckpoints.remove(posInMap);
 
-				
-				// remove older checkpoints in map
-				for (int i = 0; i < posInMap; i++) {
-					pendingCheckpoints.remove(0);
-				}
+			// remove older checkpoints in map
+			for (int i = 0; i < posInMap; i++) {
+				pendingCheckpoints.remove(0);
 			}
+
 			if (checkpointOffsets == null || checkpointOffsets.size() == 0) {
 				LOG.debug("Checkpoint state was empty.");
 				return;
 			}
-			commitOffsets(checkpointOffsets);
+			fetcher.commitSpecificOffsetsToKafka(checkpointOffsets);
 		}
 		catch (Exception e) {
 			if (running) {
@@ -511,33 +374,77 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 		}
 	}
 
-	protected abstract void commitOffsets(HashMap<KafkaTopicPartition, Long> checkpointOffsets) throws Exception;
-
-
+	// ------------------------------------------------------------------------
+	//  Kafka Consumer specific methods
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Creates the fetcher that connect to the Kafka brokers, pulls data, deserialized the
+	 * data, and emits it into the data streams.
+	 * 
+	 * @param sourceContext The source context to emit data to.
+	 * @param thisSubtaskPartitions The set of partitions that this subtask should handle.
+	 * @param watermarksPeriodic Optional, a serialized timestamp extractor / periodic watermark generator.
+	 * @param watermarksPunctuated Optional, a serialized timestamp extractor / punctuated watermark generator.
+	 * @param runtimeContext The task's runtime context.
+	 * 
+	 * @return The instantiated fetcher
+	 * 
+	 * @throws Exception The method should forward exceptions
+	 */
+	protected abstract AbstractFetcher<T, ?> createFetcher(
+			SourceContext<T> sourceContext,
+			List<KafkaTopicPartition> thisSubtaskPartitions,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			StreamingRuntimeContext runtimeContext) throws Exception;
+	
+	// ------------------------------------------------------------------------
+	//  ResultTypeQueryable methods 
+	// ------------------------------------------------------------------------
+	
 	@Override
 	public TypeInformation<T> getProducedType() {
 		return deserializer.getProducedType();
 	}
 
-	protected static <T> List<T> assignPartitions(List<T> partitions, int numConsumers, int consumerIndex) {
-		checkArgument(numConsumers > 0);
-		checkArgument(consumerIndex < numConsumers);
-
-		List<T> partitionsToSub = new ArrayList<>();
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
 
-		for (int i = 0; i < partitions.size(); i++) {
+	/**
+	 * Selects which of the given partitions should be handled by a specific consumer,
+	 * given a certain number of consumers.
+	 * 
+	 * @param allPartitions The partitions to select from
+	 * @param numConsumers The number of consumers
+	 * @param consumerIndex The index of the specific consumer
+	 * 
+	 * @return The sublist of partitions to be handled by that consumer.
+	 */
+	protected static List<KafkaTopicPartition> assignPartitions(
+			List<KafkaTopicPartition> allPartitions,
+			int numConsumers, int consumerIndex)
+	{
+		final List<KafkaTopicPartition> thisSubtaskPartitions = new ArrayList<>(
+				allPartitions.size() / numConsumers + 1);
+
+		for (int i = 0; i < allPartitions.size(); i++) {
 			if (i % numConsumers == consumerIndex) {
-				partitionsToSub.add(partitions.get(i));
+				thisSubtaskPartitions.add(allPartitions.get(i));
 			}
 		}
-		return partitionsToSub;
+		
+		return thisSubtaskPartitions;
 	}
-
+	
 	/**
-	 * Method to log partition information.
+	 * Logs the partition information in INFO level.
+	 * 
+	 * @param logger The logger to log to.
 	 * @param partitionInfos List of subscribed partitions
 	 */
-	public static void logPartitionInfo(List<KafkaTopicPartition> partitionInfos) {
+	protected static void logPartitionInfo(Logger logger, List<KafkaTopicPartition> partitionInfos) {
 		Map<String, Integer> countPerTopic = new HashMap<>();
 		for (KafkaTopicPartition partition : partitionInfos) {
 			Integer count = countPerTopic.get(partition.getTopic());
@@ -548,12 +455,13 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 			}
 			countPerTopic.put(partition.getTopic(), count);
 		}
-		StringBuilder sb = new StringBuilder();
+		StringBuilder sb = new StringBuilder(
+				"Consumer is going to read the following topics (with number of partitions): ");
+		
 		for (Map.Entry<String, Integer> e : countPerTopic.entrySet()) {
 			sb.append(e.getKey()).append(" (").append(e.getValue()).append("), ");
 		}
-		LOG.info("Consumer is going to read the following topics (with number of partitions): {}", sb.toString());
+		
+		logger.info(sb.toString());
 	}
-
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
new file mode 100644
index 0000000..594aa66
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcher.java
@@ -0,0 +1,439 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Base class for all fetchers, which implement the connections to Kafka brokers and
+ * pull records from Kafka partitions.
+ * 
+ * <p>This fetcher base class implements the logic around emitting records and tracking offsets,
+ * as well as around the optional timestamp assignment and watermark generation. 
+ * 
+ * @param <T> The type of elements deserialized from Kafka's byte records, and emitted into
+ *            the Flink data streams.
+ * @param <KPH> The type of topic/partition identifier used by Kafka in the specific version.
+ */
+public abstract class AbstractFetcher<T, KPH> {
+	
+	private static final int NO_TIMESTAMPS_WATERMARKS = 0;
+	private static final int PERIODIC_WATERMARKS = 1;
+	private static final int PUNCTUATED_WATERMARKS = 2;
+	
+	// ------------------------------------------------------------------------
+	
+	/** The source context to emit records and watermarks to */
+	private final SourceContext<T> sourceContext;
+
+	/** The lock that guarantees that record emission and state updates are atomic,
+	 * from the view of taking a checkpoint */
+	private final Object checkpointLock;
+
+	/** All partitions (and their state) that this fetcher is subscribed to */
+	private final KafkaTopicPartitionState<KPH>[] allPartitions;
+
+	/** The mode describing whether the fetcher also generates timestamps and watermarks */
+	private final int timestampWatermarkMode;
+	
+	/** Only relevant for punctuated watermarks: The current cross partition watermark */
+	private volatile long maxWatermarkSoFar = Long.MIN_VALUE;
+
+	// ------------------------------------------------------------------------
+	
+	protected AbstractFetcher(
+			SourceContext<T> sourceContext,
+			List<KafkaTopicPartition> assignedPartitions,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			StreamingRuntimeContext runtimeContext) throws Exception
+	{
+		this.sourceContext = checkNotNull(sourceContext);
+		this.checkpointLock = sourceContext.getCheckpointLock();
+		
+		// figure out what we watermark mode we will be using
+		
+		if (watermarksPeriodic == null) {
+			if (watermarksPunctuated == null) {
+				// simple case, no watermarks involved
+				timestampWatermarkMode = NO_TIMESTAMPS_WATERMARKS;
+			} else {
+				timestampWatermarkMode = PUNCTUATED_WATERMARKS;
+			}
+		} else {
+			if (watermarksPunctuated == null) {
+				timestampWatermarkMode = PERIODIC_WATERMARKS;
+			} else {
+				throw new IllegalArgumentException("Cannot have both periodic and punctuated watermarks");
+			}
+		}
+		
+		// create our partition state according to the timestamp/watermark mode 
+		this.allPartitions = initializePartitions(
+				assignedPartitions,
+				timestampWatermarkMode,
+				watermarksPeriodic, watermarksPunctuated,
+				runtimeContext.getUserCodeClassLoader());
+		
+		// if we have periodic watermarks, kick off the interval scheduler
+		if (timestampWatermarkMode == PERIODIC_WATERMARKS) {
+			KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] parts = 
+					(KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[]) allPartitions;
+			
+			PeriodicWatermarkEmitter periodicEmitter = 
+					new PeriodicWatermarkEmitter(parts, sourceContext, runtimeContext);
+			periodicEmitter.start();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets all partitions (with partition state) that this fetcher is subscribed to.
+	 *
+	 * @return All subscribed partitions.
+	 */
+	protected final KafkaTopicPartitionState<KPH>[] subscribedPartitions() {
+		return allPartitions;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Core fetcher work methods
+	// ------------------------------------------------------------------------
+
+	public abstract void runFetchLoop() throws Exception;
+	
+	public abstract void cancel();
+
+	// ------------------------------------------------------------------------
+	//  Kafka version specifics
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Creates the Kafka version specific representation of the given
+	 * topic partition.
+	 * 
+	 * @param partition The Flink representation of the Kafka topic partition.
+	 * @return The specific Kafka representation of the Kafka topic partition.
+	 */
+	public abstract KPH createKafkaPartitionHandle(KafkaTopicPartition partition);
+
+	/**
+	 * Commits the given partition offsets to the Kafka brokers (or to ZooKeeper for
+	 * older Kafka versions).
+	 * 
+	 * @param offsets The offsets to commit to Kafka.
+	 * @throws Exception This method forwards exceptions.
+	 */
+	public abstract void commitSpecificOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception;
+	
+	// ------------------------------------------------------------------------
+	//  snapshot and restore the state
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Takes a snapshot of the partition offsets.
+	 * 
+	 * <p>Important: This method mus be called under the checkpoint lock.
+	 * 
+	 * @return A map from partition to current offset.
+	 */
+	public HashMap<KafkaTopicPartition, Long> snapshotCurrentState() {
+		// this method assumes that the checkpoint lock is held
+		assert Thread.holdsLock(checkpointLock);
+
+		HashMap<KafkaTopicPartition, Long> state = new HashMap<>(allPartitions.length);
+		for (KafkaTopicPartitionState<?> partition : subscribedPartitions()) {
+			if (partition.isOffsetDefined()) {
+				state.put(partition.getKafkaTopicPartition(), partition.getOffset());
+			}
+		}
+		return state;
+	}
+
+	/**
+	 * Restores the partition offsets.
+	 * 
+	 * @param snapshotState The offsets for the partitions 
+	 */
+	public void restoreOffsets(HashMap<KafkaTopicPartition, Long> snapshotState) {
+		for (KafkaTopicPartitionState<?> partition : allPartitions) {
+			Long offset = snapshotState.get(partition.getKafkaTopicPartition());
+			if (offset != null) {
+				partition.setOffset(offset);
+			}
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  emitting records
+	// ------------------------------------------------------------------------
+
+	/**
+	 * 
+	 * <p>Implementation Note: This method is kept brief to be JIT inlining friendly.
+	 * That makes the fast path efficient, the extended paths are called as separate methods.
+	 * 
+	 * @param record The record to emit
+	 * @param partitionState The state of the Kafka partition from which the record was fetched
+	 * @param offset The offset from which the record was fetched
+	 */
+	protected final void emitRecord(T record, KafkaTopicPartitionState<KPH> partitionState, long offset) {
+		if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS) {
+			// fast path logic, in case there are no watermarks
+
+			// emit the record, using the checkpoint lock to guarantee
+			// atomicity of record emission and offset state update
+			synchronized (checkpointLock) {
+				sourceContext.collect(record);
+				partitionState.setOffset(offset);
+			}
+		}
+		else if (timestampWatermarkMode == PERIODIC_WATERMARKS) {
+			emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset);
+		}
+		else {
+			emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset);
+		}
+	}
+
+	/**
+	 * Record emission, if a timestamp will be attached from an assigner that is
+	 * also a periodic watermark generator.
+	 */
+	private void emitRecordWithTimestampAndPeriodicWatermark(
+			T record, KafkaTopicPartitionState<KPH> partitionState, long offset)
+	{
+		@SuppressWarnings("unchecked")
+		final KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH> withWatermarksState =
+				(KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH>) partitionState;
+
+		// extract timestamp - this accesses/modifies the per-partition state inside the
+		// watermark generator instance, so we need to lock the access on the
+		// partition state. concurrent access can happen from the periodic emitter
+		final long timestamp;
+		//noinspection SynchronizationOnLocalVariableOrMethodParameter
+		synchronized (withWatermarksState) {
+			timestamp = withWatermarksState.getTimestampForRecord(record);
+		}
+
+		// emit the record with timestamp, using the usual checkpoint lock to guarantee
+		// atomicity of record emission and offset state update 
+		synchronized (checkpointLock) {
+			sourceContext.collectWithTimestamp(record, timestamp);
+			partitionState.setOffset(offset);
+		}
+	}
+
+	/**
+	 * Record emission, if a timestamp will be attached from an assigner that is
+	 * also a punctuated watermark generator.
+	 */
+	private void emitRecordWithTimestampAndPunctuatedWatermark(
+			T record, KafkaTopicPartitionState<KPH> partitionState, long offset)
+	{
+		@SuppressWarnings("unchecked")
+		final KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH> withWatermarksState =
+				(KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH>) partitionState;
+
+		// only one thread ever works on accessing timestamps and watermarks
+		// from the punctuated extractor
+		final long timestamp = withWatermarksState.getTimestampForRecord(record);
+		final Watermark newWatermark = withWatermarksState.checkAndGetNewWatermark(record, timestamp);
+			
+		// emit the record with timestamp, using the usual checkpoint lock to guarantee
+		// atomicity of record emission and offset state update 
+		synchronized (checkpointLock) {
+			sourceContext.collectWithTimestamp(record, timestamp);
+			partitionState.setOffset(offset);
+		}
+		
+		// if we also have a new per-partition watermark, check if that is also a
+		// new cross-partition watermark
+		if (newWatermark != null) {
+			updateMinPunctuatedWatermark(newWatermark);
+		}
+	}
+	/**
+	 *Checks whether a new per-partition watermark is also a new cross-partition watermark.
+	 */
+	private void updateMinPunctuatedWatermark(Watermark nextWatermark) {
+		if (nextWatermark.getTimestamp() > maxWatermarkSoFar) {
+			long newMin = Long.MAX_VALUE;
+			
+			for (KafkaTopicPartitionState<?> state : allPartitions) {
+				@SuppressWarnings("unchecked")
+				final KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH> withWatermarksState =
+						(KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH>) state;
+				
+				newMin = Math.min(newMin, withWatermarksState.getCurrentPartitionWatermark());
+			}
+			
+			// double-check locking pattern
+			if (newMin > maxWatermarkSoFar) {
+				synchronized (checkpointLock) {
+					if (newMin > maxWatermarkSoFar) {
+						maxWatermarkSoFar = newMin;
+						sourceContext.emitWatermark(new Watermark(newMin));
+					}
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Utility method that takes the topic partitions and creates the topic partition state
+	 * holders. If a watermark generator per partition exists, this will also initialize those.
+	 */
+	private KafkaTopicPartitionState<KPH>[] initializePartitions(
+			List<KafkaTopicPartition> assignedPartitions,
+			int timestampWatermarkMode,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			ClassLoader userCodeClassLoader)
+		throws IOException, ClassNotFoundException
+	{
+		@SuppressWarnings("unchecked")
+		KafkaTopicPartitionState<KPH>[] partitions =
+				(KafkaTopicPartitionState<KPH>[]) new KafkaTopicPartitionState<?>[assignedPartitions.size()];
+
+		int pos = 0;
+		for (KafkaTopicPartition partition : assignedPartitions) {
+			// create the kafka version specific partition handle
+			KPH kafkaHandle = createKafkaPartitionHandle(partition);
+			
+			// create the partition state
+			KafkaTopicPartitionState<KPH> partitionState;
+			switch (timestampWatermarkMode) {
+				case NO_TIMESTAMPS_WATERMARKS:
+					partitionState = new KafkaTopicPartitionState<>(partition, kafkaHandle);
+					break;
+				case PERIODIC_WATERMARKS: {
+					AssignerWithPeriodicWatermarks<T> assignerInstance =
+							watermarksPeriodic.deserializeValue(userCodeClassLoader);
+					partitionState = new KafkaTopicPartitionStateWithPeriodicWatermarks<>(
+							partition, kafkaHandle, assignerInstance);
+					break;
+				}
+					
+				case PUNCTUATED_WATERMARKS: {
+					AssignerWithPunctuatedWatermarks<T> assignerInstance =
+							watermarksPunctuated.deserializeValue(userCodeClassLoader);
+					partitionState = new KafkaTopicPartitionStateWithPunctuatedWatermarks<>(
+							partition, kafkaHandle, assignerInstance);
+					break;
+				}
+				default:
+					// cannot happen, add this as a guard for the future
+					throw new RuntimeException();
+			}
+
+			partitions[pos++] = partitionState;
+		}
+		
+		return partitions;
+	}
+	
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * The periodic watermark emitter. In its given interval, it checks all partitions for
+	 * the current event time watermark, and possibly emits the next watermark.
+	 */
+	private static class PeriodicWatermarkEmitter implements Triggerable {
+
+		private final KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] allPartitions;
+		
+		private final SourceContext<?> emitter;
+		
+		private final StreamingRuntimeContext triggerContext;
+
+		private final long interval;
+		
+		private long lastWatermarkTimestamp;
+		
+		//-------------------------------------------------
+
+		PeriodicWatermarkEmitter(
+				KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?>[] allPartitions,
+				SourceContext<?> emitter,
+				StreamingRuntimeContext runtimeContext)
+		{
+			this.allPartitions = checkNotNull(allPartitions);
+			this.emitter = checkNotNull(emitter);
+			this.triggerContext = checkNotNull(runtimeContext);
+			this.interval = runtimeContext.getExecutionConfig().getAutoWatermarkInterval();
+			this.lastWatermarkTimestamp = Long.MIN_VALUE;
+		}
+
+		//-------------------------------------------------
+		
+		public void start() {
+			triggerContext.registerTimer(System.currentTimeMillis() + interval, this);
+		}
+		
+		@Override
+		public void trigger(long timestamp) throws Exception {
+			// sanity check
+			assert Thread.holdsLock(emitter.getCheckpointLock());
+			
+			long minAcrossAll = Long.MAX_VALUE;
+			for (KafkaTopicPartitionStateWithPeriodicWatermarks<?, ?> state : allPartitions) {
+				
+				// we access the current watermark for the periodic assigners under the state
+				// lock, to prevent concurrent modification to any internal variables
+				final long curr;
+				//noinspection SynchronizationOnLocalVariableOrMethodParameter
+				synchronized (state) {
+					curr = state.getCurrentWatermarkTimestamp();
+				}
+				
+				minAcrossAll = Math.min(minAcrossAll, curr);
+			}
+			
+			// emit next watermark, if there is one
+			if (minAcrossAll > lastWatermarkTimestamp) {
+				lastWatermarkTimestamp = minAcrossAll;
+				emitter.emitWatermark(new Watermark(minAcrossAll));
+			}
+			
+			// schedule the next watermark
+			triggerContext.registerTimer(System.currentTimeMillis() + interval, this);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
new file mode 100644
index 0000000..9a0e4e3
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ExceptionProxy.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import javax.annotation.Nullable;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * 
+ */
+public class ExceptionProxy {
+	
+	/** The thread that should be interrupted when an exception occurs */
+	private final Thread toInterrupt;
+	
+	/** The exception to throw */ 
+	private final AtomicReference<Throwable> exception;
+
+	/**
+	 * 
+	 * @param toInterrupt The thread to interrupt upon an exception. May be null.
+	 */
+	public ExceptionProxy(@Nullable Thread toInterrupt) {
+		this.toInterrupt = toInterrupt;
+		this.exception = new AtomicReference<>();
+	}
+	
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Sets the exception occurred and interrupts the target thread,
+	 * if no other exception has occurred so far.
+	 * 
+	 * @param t The exception that occurred
+	 */
+	public void reportError(Throwable t) {
+		// set the exception, if it is the first
+		if (exception.compareAndSet(null, t) && toInterrupt != null) {
+			toInterrupt.interrupt();
+		}
+	}
+	
+	public void checkAndThrowException() throws Exception {
+		Throwable t = exception.get();
+		if (t != null) {
+			if (t instanceof Exception) {
+				throw (Exception) t;
+			}
+			else if (t instanceof Error) {
+				throw (Error) t;
+			}
+			else {
+				throw new Exception(t);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java
deleted file mode 100644
index 11a392a..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import java.io.Serializable;
-
-public class KafkaPartitionState implements Serializable {
-
-	private static final long serialVersionUID = 722083576322742328L;
-
-	private final int partitionID;
-	private long offset;
-
-	private long maxTimestamp = Long.MIN_VALUE;
-	private boolean isActive = false;
-
-	public KafkaPartitionState(int id, long offset) {
-		this.partitionID = id;
-		this.offset = offset;
-	}
-
-	public void setOffset(long offset) {
-		this.offset = offset;
-	}
-
-	public void setActive(boolean isActive) {
-		this.isActive = isActive;
-	}
-
-	public void setMaxTimestamp(long timestamp) {
-		maxTimestamp = timestamp;
-	}
-
-	public int getPartition() {
-		return partitionID;
-	}
-
-	public boolean isActive() {
-		return isActive;
-	}
-
-	public long getMaxTimestamp() {
-		return maxTimestamp;
-	}
-
-	public long getOffset() {
-		return offset;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
index aea14cf..c68fe28 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java
@@ -24,14 +24,20 @@ import java.util.Map;
 
 import static java.util.Objects.requireNonNull;
 
-
 /**
- * A serializable representation of a kafka topic and a partition.
- * Used as an operator state for the Kafka consumer
+ * Flink's description of a partition in a Kafka topic.
+ * Serializable, and common across all Kafka consumer subclasses (0.8, 0.9, ...)
+ * 
+ * <p>Note: This class must not change in its structure, because it would change the
+ * serialization format and make previous savepoints unreadable.
  */
-public class KafkaTopicPartition implements Serializable {
+public final class KafkaTopicPartition implements Serializable {
 
+	/** THIS SERIAL VERSION UID MUST NOT CHANGE, BECAUSE IT WOULD BREAK
+	 * READING OLD SERIALIZED INSTANCES FROM SAVEPOINTS */
 	private static final long serialVersionUID = 722083576322742325L;
+	
+	// ------------------------------------------------------------------------
 
 	private final String topic;
 	private final int partition;
@@ -43,6 +49,8 @@ public class KafkaTopicPartition implements Serializable {
 		this.cachedHash = 31 * topic.hashCode() + partition;
 	}
 
+	// ------------------------------------------------------------------------
+	
 	public String getTopic() {
 		return topic;
 	}
@@ -51,6 +59,8 @@ public class KafkaTopicPartition implements Serializable {
 		return partition;
 	}
 
+	// ------------------------------------------------------------------------
+	
 	@Override
 	public String toString() {
 		return "KafkaTopicPartition{" +
@@ -64,25 +74,23 @@ public class KafkaTopicPartition implements Serializable {
 		if (this == o) {
 			return true;
 		}
-		if (!(o instanceof KafkaTopicPartition)) {
-			return false;
+		else if (o instanceof KafkaTopicPartition) {
+			KafkaTopicPartition that = (KafkaTopicPartition) o;
+			return this.partition == that.partition && this.topic.equals(that.topic);
 		}
-
-		KafkaTopicPartition that = (KafkaTopicPartition) o;
-
-		if (partition != that.partition) {
+		else {
 			return false;
 		}
-		return topic.equals(that.topic);
 	}
 
 	@Override
 	public int hashCode() {
 		return cachedHash;
 	}
-
-
-	// ------------------- Utilities -------------------------------------
+	
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
 
 	public static String toString(Map<KafkaTopicPartition, Long> map) {
 		StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
new file mode 100644
index 0000000..36612a4
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionState.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+/**
+ * The state that the Flink Kafka Consumer holds for each Kafka partition.
+ * Includes the Kafka descriptor for partitions.
+ * 
+ * <p>This class describes the most basic state (only the offset), subclasses
+ * define more elaborate state, containing current watermarks and timestamp
+ * extractors.
+ * 
+ * @param <KPH> The type of the Kafka partition descriptor, which varies across Kafka versions.
+ */
+public class KafkaTopicPartitionState<KPH> {
+
+	/** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid),
+	 * and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */
+	public static final long OFFSET_NOT_SET = -915623761776L;
+	
+	// ------------------------------------------------------------------------
+
+	/** The Flink description of a Kafka partition */
+	private final KafkaTopicPartition partition;
+
+	/** The Kafka description of a Kafka partition (varies across different Kafka versions) */
+	private final KPH kafkaPartitionHandle;
+	
+	/** The offset within the Kafka partition that we already processed */
+	private volatile long offset;
+
+	// ------------------------------------------------------------------------
+	
+	public KafkaTopicPartitionState(KafkaTopicPartition partition, KPH kafkaPartitionHandle) {
+		this.partition = partition;
+		this.kafkaPartitionHandle = kafkaPartitionHandle;
+		this.offset = OFFSET_NOT_SET;
+	}
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets Flink's descriptor for the Kafka Partition.
+	 * @return The Flink partition descriptor.
+	 */
+	public final KafkaTopicPartition getKafkaTopicPartition() {
+		return partition;
+	}
+
+	/**
+	 * Gets Kafka's descriptor for the Kafka Partition.
+	 * @return The Kafka partition descriptor.
+	 */
+	public final KPH getKafkaPartitionHandle() {
+		return kafkaPartitionHandle;
+	}
+
+	public final String getTopic() {
+		return partition.getTopic();
+	}
+
+	public final int getPartition() {
+		return partition.getPartition();
+	}
+
+	/**
+	 * The current offset in the partition. This refers to the offset last element that
+	 * we retrieved and emitted successfully. It is the offset that should be stored in
+	 * a checkpoint.
+	 */
+	public final long getOffset() {
+		return offset;
+	}
+
+	public final void setOffset(long offset) {
+		this.offset = offset;
+	}
+	
+	public final boolean isOffsetDefined() {
+		return offset != OFFSET_NOT_SET;
+	}
+	
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return "Partition: " + partition + ", KafkaPartitionHandle=" + kafkaPartitionHandle
+				+ ", offset=" + (isOffsetDefined() ? String.valueOf(offset) : "(not set)");
+	}
+}


[11/14] flink git commit: [FLINK-3375] [kafka connector] Rework/simplify Kafka Connector and have a WatermarkExtractor object per partition

Posted by se...@apache.org.
[FLINK-3375] [kafka connector] Rework/simplify Kafka Connector and have a WatermarkExtractor object per partition


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

Branch: refs/heads/master
Commit: 3c93103d1476cb05ec0c018bfa6876e4ecad38e8
Parents: 0ac1549
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Apr 6 23:21:17 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 .../connectors/kafka/FlinkKafkaConsumer08.java  | 358 ++------
 .../connectors/kafka/internals/Fetcher.java     |  72 --
 .../kafka/internals/Kafka08Fetcher.java         | 446 +++++++++
 .../kafka/internals/KillerWatchDog.java         |  62 ++
 .../kafka/internals/LegacyFetcher.java          | 896 -------------------
 .../kafka/internals/OffsetHandler.java          |  55 --
 .../kafka/internals/PartitionInfoFetcher.java   |  66 ++
 .../kafka/internals/PartitionerWrapper.java     |  49 -
 .../internals/PeriodicOffsetCommitter.java      |  85 ++
 .../kafka/internals/SimpleConsumerThread.java   | 504 +++++++++++
 .../kafka/internals/ZookeeperOffsetHandler.java |  58 +-
 .../connectors/kafka/Kafka08ITCase.java         | 176 ++--
 .../connectors/kafka/KafkaConsumer08Test.java   |  90 ++
 .../connectors/kafka/KafkaConsumerTest.java     | 156 ----
 .../kafka/KafkaShortRetention08ITCase.java      |   3 +-
 .../internals/ZookeeperOffsetHandlerTest.java   |  56 --
 .../src/test/resources/log4j-test.properties    |   5 +-
 .../connectors/kafka/FlinkKafkaConsumer09.java  | 398 ++------
 .../kafka/internal/Kafka09Fetcher.java          | 311 +++++++
 .../connectors/kafka/Kafka09ITCase.java         |  21 +-
 .../connectors/kafka/KafkaProducerTest.java     |   8 +-
 .../kafka/KafkaShortRetention09ITCase.java      |   1 +
 .../src/test/resources/log4j-test.properties    |   5 +-
 .../kafka/FlinkKafkaConsumerBase.java           | 668 ++++++--------
 .../kafka/internals/AbstractFetcher.java        | 439 +++++++++
 .../kafka/internals/ExceptionProxy.java         |  73 ++
 .../kafka/internals/KafkaPartitionState.java    |  65 --
 .../kafka/internals/KafkaTopicPartition.java    |  36 +-
 .../internals/KafkaTopicPartitionState.java     | 105 +++
 ...picPartitionStateWithPeriodicWatermarks.java |  71 ++
 ...cPartitionStateWithPunctuatedWatermarks.java |  84 ++
 .../kafka/partitioner/KafkaPartitioner.java     |   2 +-
 .../connectors/kafka/util/KafkaUtils.java       |  13 +-
 .../kafka/FlinkKafkaConsumerBaseTest.java       | 222 +++++
 .../KafkaConsumerPartitionAssignmentTest.java   |  96 +-
 .../connectors/kafka/KafkaConsumerTestBase.java | 175 ++--
 .../connectors/kafka/KafkaProducerTestBase.java |   8 +-
 .../kafka/KafkaShortRetentionTestBase.java      |  45 +-
 .../internals/KafkaTopicPartitionTest.java      |  57 ++
 .../testutils/JobManagerCommunicationUtils.java |  49 +-
 .../kafka/testutils/MockRuntimeContext.java     |  26 +-
 .../AssignerWithPeriodicWatermarks.java         |   3 +
 .../AssignerWithPunctuatedWatermarks.java       |   3 +
 43 files changed, 3407 insertions(+), 2714 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
index 4748781..48cc461 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import kafka.api.OffsetRequest;
 import kafka.cluster.Broker;
 import kafka.common.ErrorMapping;
 import kafka.javaapi.PartitionMetadata;
@@ -24,40 +25,32 @@ import kafka.javaapi.TopicMetadata;
 import kafka.javaapi.TopicMetadataRequest;
 import kafka.javaapi.consumer.SimpleConsumer;
 
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.internals.Fetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaPartitionState;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.Kafka08Fetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.OffsetHandler;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader;
-import org.apache.flink.streaming.connectors.kafka.internals.ZookeeperOffsetHandler;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 import org.apache.flink.util.NetUtils;
+import org.apache.flink.util.SerializedValue;
+
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.common.Node;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 
-import static java.util.Objects.requireNonNull;
 import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.getIntFromConfig;
 import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.getLongFromConfig;
-
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
@@ -102,12 +95,8 @@ import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.getLon
  * reach the Kafka brokers or ZooKeeper.</p>
  */
 public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
-	
-	// ------------------------------------------------------------------------
-	
+
 	private static final long serialVersionUID = -6272159445203409112L;
-	
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer08.class);
 
 	/** Configuration key for the number of retries for getting the partition info */
 	public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry";
@@ -115,30 +104,16 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 	/** Default number of retries for getting the partition info. One retry means going through the full list of brokers */
 	public static final int DEFAULT_GET_PARTITIONS_RETRIES = 3;
 
-	
-	// ------  Configuration of the Consumer -------
+	// ------------------------------------------------------------------------
 
-	/** Initial list of topics and partitions to consume  */
-	private final List<KafkaTopicPartition> partitionInfos;
-	
 	/** The properties to parametrize the Kafka consumer and ZooKeeper client */ 
-	private final Properties props;
+	private final Properties kafkaProperties;
 
+	/** The behavior when encountering an invalid offset (see {@link OffsetRequest}) */
+	private final long invalidOffsetBehavior;
 
-	// ------  Runtime State  -------
-	
-	/** The fetcher used to pull data from the Kafka brokers */
-	private transient Fetcher fetcher;
-	
-	/** The committer that persists the committed offsets */
-	private transient OffsetHandler offsetHandler;
-	
-	/** The partitions actually handled by this consumer at runtime */
-	private transient List<KafkaTopicPartition> subscribedPartitions;
-
-	/** The latest offsets that have been committed to Kafka or ZooKeeper. These are never
-	 * newer then the last offsets (Flink's internal view is fresher) */
-	private transient HashMap<KafkaTopicPartition, Long> committedOffsets;
+	/** The interval in which to automatically commit (-1 if deactivated) */
+	private final long autoCommitInterval;
 
 	// ------------------------------------------------------------------------
 
@@ -202,287 +177,49 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 	 *           The properties that are used to configure both the fetcher and the offset handler.
 	 */
 	public FlinkKafkaConsumer08(List<String> topics, KeyedDeserializationSchema<T> deserializer, Properties props) {
-		super(deserializer, props);
+		super(deserializer);
 
-		requireNonNull(topics, "topics");
-		this.props = requireNonNull(props, "props");
+		checkNotNull(topics, "topics");
+		this.kafkaProperties = checkNotNull(props, "props");
 
 		// validate the zookeeper properties
 		validateZooKeeperConfig(props);
 
+		this.invalidOffsetBehavior = getInvalidOffsetBehavior(props);
+		this.autoCommitInterval = getLongFromConfig(props, "auto.commit.interval.ms", 60000);
+
 		// Connect to a broker to get the partitions for all topics
-		this.partitionInfos = KafkaTopicPartition.dropLeaderData(getPartitionsForTopic(topics, props));
+		List<KafkaTopicPartition> partitionInfos = 
+				KafkaTopicPartition.dropLeaderData(getPartitionsForTopic(topics, props));
 
 		if (partitionInfos.size() == 0) {
-			throw new RuntimeException("Unable to retrieve any partitions for the requested topics " + topics.toString() + "." +
-					"Please check previous log entries");
-		}
-
-		if (LOG.isInfoEnabled()) {
-			logPartitionInfo(partitionInfos);
+			throw new RuntimeException(
+					"Unable to retrieve any partitions for the requested topics " + topics + 
+							". Please check previous log entries");
 		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Source life cycle
-	// ------------------------------------------------------------------------
 
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-		
-		final int numConsumers = getRuntimeContext().getNumberOfParallelSubtasks();
-		final int thisConsumerIndex = getRuntimeContext().getIndexOfThisSubtask();
-		
-		// pick which partitions we work on
-		subscribedPartitions = assignPartitions(this.partitionInfos, numConsumers, thisConsumerIndex);
-		
 		if (LOG.isInfoEnabled()) {
-			LOG.info("Kafka consumer {} will read partitions {} out of partitions {}",
-					thisConsumerIndex, KafkaTopicPartition.toString(subscribedPartitions), this.partitionInfos.size());
-		}
-
-		// we leave the fetcher as null, if we have no partitions
-		if (subscribedPartitions.isEmpty()) {
-			LOG.info("Kafka consumer {} has no partitions (empty source)", thisConsumerIndex);
-			this.fetcher = null; // fetcher remains null
-			return;
-		}
-
-		// offset handling
-		offsetHandler = new ZookeeperOffsetHandler(props);
-
-		committedOffsets = new HashMap<>();
-
-		// initially load the map with "offset not set", last max read timestamp set to Long.MIN_VALUE
-		// and mark the partition as in-active, until we receive the first element
-		Map<KafkaTopicPartition, KafkaPartitionState> subscribedPartitionsWithOffsets =
-			new HashMap<>(subscribedPartitions.size());
-		for(KafkaTopicPartition ktp: subscribedPartitions) {
-			subscribedPartitionsWithOffsets.put(ktp,
-				new KafkaPartitionState(ktp.getPartition(), FlinkKafkaConsumerBase.OFFSET_NOT_SET));
-		}
-
-		// seek to last known pos, from restore request
-		if (restoreToOffset != null) {
-			if (LOG.isInfoEnabled()) {
-				LOG.info("Consumer {} is restored from previous checkpoint: {}",
-						thisConsumerIndex, KafkaTopicPartition.toString(restoreToOffset));
-			}
-			// initialize offsets with restored state
-			this.partitionState = restoreInfoFromCheckpoint();
-			subscribedPartitionsWithOffsets.putAll(partitionState);
-			restoreToOffset = null;
-		}
-		else {
-			// start with empty partition state
-			partitionState = new HashMap<>();
-
-			// no restore request: overwrite offsets.
-			for(Map.Entry<KafkaTopicPartition, Long> offsetInfo: offsetHandler.getOffsets(subscribedPartitions).entrySet()) {
-				KafkaTopicPartition key = offsetInfo.getKey();
-				subscribedPartitionsWithOffsets.put(key,
-					new KafkaPartitionState(key.getPartition(), offsetInfo.getValue()));
-			}
-		}
-		if(subscribedPartitionsWithOffsets.size() != subscribedPartitions.size()) {
-			throw new IllegalStateException("The subscribed partitions map has more entries than the subscribed partitions " +
-					"list: " + subscribedPartitionsWithOffsets.size() + "," + subscribedPartitions.size());
-		}
-
-		// create fetcher
-		fetcher = new LegacyFetcher<T>(this, subscribedPartitionsWithOffsets, props,
-				getRuntimeContext().getTaskName(), getRuntimeContext().getUserCodeClassLoader());
-	}
-
-	@Override
-	public void run(SourceContext<T> sourceContext) throws Exception {
-		if (fetcher != null) {
-			StreamingRuntimeContext streamingRuntimeContext = (StreamingRuntimeContext) getRuntimeContext();
-
-			// if we have a non-checkpointed source, start a thread which periodically commits
-			// the current offset into ZK.
-
-			PeriodicOffsetCommitter<T> offsetCommitter = null;
-			if (!streamingRuntimeContext.isCheckpointingEnabled()) {
-				// we use Kafka's own configuration parameter key for this.
-				// Note that the default configuration value in Kafka is 60 * 1000, so we use the same here.
-				long commitInterval = getLongFromConfig(props, "auto.commit.interval.ms", 60000);
-				offsetCommitter = new PeriodicOffsetCommitter<>(commitInterval, this);
-				offsetCommitter.setDaemon(true);
-				offsetCommitter.start();
-				LOG.info("Starting periodic offset committer, with commit interval of {}ms", commitInterval);
-			}
-
-			try {
-				fetcher.run(sourceContext, deserializer, partitionState);
-			} finally {
-				if (offsetCommitter != null) {
-					offsetCommitter.close();
-					try {
-						offsetCommitter.join();
-					} catch(InterruptedException ie) {
-						// ignore interrupt
-					}
-				}
-			}
-		}
-		else {
-			// this source never completes, so emit a Long.MAX_VALUE watermark
-			// to not block watermark forwarding
-			sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
-
-			final Object waitLock = new Object();
-			while (running) {
-				// wait until we are canceled
-				try {
-					//noinspection SynchronizationOnLocalVariableOrMethodParameter
-					synchronized (waitLock) {
-						waitLock.wait();
-					}
-				}
-				catch (InterruptedException e) {
-					// do nothing, check our "running" status
-				}
-			}
-		}
-		
-		// close the context after the work was done. this can actually only
-		// happen when the fetcher decides to stop fetching
-		sourceContext.close();
-	}
-
-	@Override
-	public void cancel() {
-		// set ourselves as not running
-		running = false;
-		
-		// close the fetcher to interrupt any work
-		Fetcher fetcher = this.fetcher;
-		this.fetcher = null;
-		if (fetcher != null) {
-			try {
-				fetcher.close();
-			}
-			catch (IOException e) {
-				LOG.warn("Error while closing Kafka connector data fetcher", e);
-			}
+			logPartitionInfo(LOG, partitionInfos);
 		}
-		
-		OffsetHandler offsetHandler = this.offsetHandler;
-		this.offsetHandler = null;
-		if (offsetHandler != null) {
-			try {
-				offsetHandler.close();
-			}
-			catch (IOException e) {
-				LOG.warn("Error while closing Kafka connector offset handler", e);
-			}
-		}
-	}
 
-	@Override
-	public void close() throws Exception {
-		cancel();
-		super.close();
+		setSubscribedPartitions(partitionInfos);
 	}
 
-	// ------------------------------------------------------------------------
-	//  Checkpoint and restore
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Utility method to commit offsets.
-	 *
-	 * @param toCommit the offsets to commit
-	 * @throws Exception
-	 */
 	@Override
-	protected void commitOffsets(HashMap<KafkaTopicPartition, Long> toCommit) throws Exception {
-		Map<KafkaTopicPartition, Long> offsetsToCommit = new HashMap<>();
-		for (KafkaTopicPartition tp : this.subscribedPartitions) {
-			Long offset = toCommit.get(tp);
-			if(offset == null) {
-				// There was no data ever consumed from this topic, that's why there is no entry
-				// for this topicPartition in the map.
-				continue;
-			}
-			Long lastCommitted = this.committedOffsets.get(tp);
-			if (lastCommitted == null) {
-				lastCommitted = OFFSET_NOT_SET;
-			}
-			if (offset != OFFSET_NOT_SET) {
-				if (offset > lastCommitted) {
-					offsetsToCommit.put(tp, offset);
-					this.committedOffsets.put(tp, offset);
-					LOG.debug("Committing offset {} for partition {}", offset, tp);
-				} else {
-					LOG.debug("Ignoring offset {} for partition {} because it is already committed", offset, tp);
-				}
-			}
-		}
-
-		if (LOG.isDebugEnabled() && offsetsToCommit.size() > 0) {
-			LOG.debug("Committing offsets {} to Zookeeper", KafkaTopicPartition.toString(offsetsToCommit));
-		}
-
-		this.offsetHandler.commit(offsetsToCommit);
+	protected AbstractFetcher<T, ?> createFetcher(
+			SourceContext<T> sourceContext,
+			List<KafkaTopicPartition> thisSubtaskPartitions,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			StreamingRuntimeContext runtimeContext) throws Exception {
+
+		return new Kafka08Fetcher<>(sourceContext, thisSubtaskPartitions,
+				watermarksPeriodic, watermarksPunctuated,
+				runtimeContext, deserializer, kafkaProperties,
+				invalidOffsetBehavior, autoCommitInterval);
 	}
 
 	// ------------------------------------------------------------------------
-	//  Miscellaneous utilities 
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Thread to periodically commit the current read offset into Zookeeper.
-	 */
-	private static class PeriodicOffsetCommitter<T> extends Thread {
-		private final long commitInterval;
-		private final FlinkKafkaConsumer08<T> consumer;
-		private volatile boolean running = true;
-
-		PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer08<T> consumer) {
-			this.commitInterval = commitInterval;
-			this.consumer = consumer;
-		}
-
-		@Override
-		public void run() {
-			try {
-
-				while (running) {
-					try {
-						Thread.sleep(commitInterval);
-						//  ------------  commit current offsets ----------------
-
-						// create copy a deep copy of the current offsets
-						@SuppressWarnings("unchecked")
-						HashMap<KafkaTopicPartition, Long> currentOffsets = new HashMap<>(consumer.partitionState.size());
-						for (Map.Entry<KafkaTopicPartition, KafkaPartitionState> entry: consumer.partitionState.entrySet()) {
-							currentOffsets.put(entry.getKey(), entry.getValue().getOffset());
-						}
-						consumer.commitOffsets(currentOffsets);
-					} catch (InterruptedException e) {
-						if (running) {
-							// throw unexpected interruption
-							throw e;
-						}
-					}
-				}
-			} catch (Throwable t) {
-				LOG.warn("Periodic checkpoint committer is stopping the fetcher because of an error", t);
-				consumer.fetcher.stopWithError(t);
-			}
-		}
-
-		public void close() {
-			this.running = false;
-			this.interrupt();
-		}
-	}
-
-
-	// ------------------------------------------------------------------------
 	//  Kafka / ZooKeeper communication utilities
 	// ------------------------------------------------------------------------
 
@@ -492,11 +229,11 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 	 * @param topics The name of the topics.
 	 * @param properties The properties for the Kafka Consumer that is used to query the partitions for the topic. 
 	 */
-	public static List<KafkaTopicPartitionLeader> getPartitionsForTopic(final List<String> topics, final Properties properties) {
+	public static List<KafkaTopicPartitionLeader> getPartitionsForTopic(List<String> topics, Properties properties) {
 		String seedBrokersConfString = properties.getProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
 		final int numRetries = getIntFromConfig(properties, GET_PARTITIONS_RETRIES_KEY, DEFAULT_GET_PARTITIONS_RETRIES);
 
-		requireNonNull(seedBrokersConfString, "Configuration property " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + " not set");
+		checkNotNull(seedBrokersConfString, "Configuration property %s not set", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
 		String[] seedBrokers = seedBrokersConfString.split(",");
 		List<KafkaTopicPartitionLeader> partitions = new ArrayList<>();
 
@@ -605,4 +342,17 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 			throw new IllegalArgumentException("Property 'zookeeper.connection.timeout.ms' is not a valid integer");
 		}
 	}
+
+	private static long getInvalidOffsetBehavior(Properties config) {
+		final String val = config.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "largest");
+		if (val.equals("none")) {
+			throw new IllegalArgumentException("Cannot use '" + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG
+					+ "' value 'none'. Possible values: 'latest', 'largest', or 'earliest'.");
+		}
+		else if (val.equals("largest") || val.equals("latest")) { // largest is kafka 0.8, latest is kafka 0.9
+			return OffsetRequest.LatestTime();
+		} else {
+			return OffsetRequest.EarliestTime();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
deleted file mode 100644
index f86687e..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-
-import java.io.IOException;
-import java.util.HashMap;
-
-/**
- * A fetcher pulls data from Kafka, from a fix set of partitions.
- * The fetcher supports "seeking" inside the partitions, i.e., moving to a different offset.
- */
-public interface Fetcher {
-
-	/**
-	 * Closes the fetcher. This will stop any operation in the
-	 * {@link #run(SourceFunction.SourceContext, KeyedDeserializationSchema, HashMap)} method and eventually
-	 * close underlying connections and release all resources.
-	 */
-	void close() throws IOException;
-
-	/**
-	 * Starts fetch data from Kafka and emitting it into the stream.
-	 * 
-	 * <p>To provide exactly once guarantees, the fetcher needs emit a record and update the last
-	 * consumed offset in one atomic operation. This is done in the
-	 * {@link org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase#processElement(SourceFunction.SourceContext, KafkaTopicPartition, Object, long)}
-	 * which is called from within the {@link SourceFunction.SourceContext#getCheckpointLock()}, as shown below:</p>
-	 * <pre>{@code
-	 * 
-	 * while (running) {
-	 *     T next = ...
-	 *     long offset = ...
-	 *     int partition = ...
-	 *     synchronized (sourceContext.getCheckpointLock()) {
-	 *         processElement(sourceContext, partition, next, offset)
-	 *     }
-	 * }
-	 * }</pre>
-	 *
-	 * @param <T> The type of elements produced by the fetcher and emitted to the source context.
-	 * @param sourceContext The source context to emit elements to.
-	 * @param valueDeserializer The deserializer to decode the raw values with.
-	 * @param lastOffsets The map into which to store the offsets for which elements are emitted (operator state)
-	 */
-	<T> void run(SourceFunction.SourceContext<T> sourceContext, KeyedDeserializationSchema<T> valueDeserializer,
-				HashMap<KafkaTopicPartition, KafkaPartitionState> lastOffsets) throws Exception;
-
-	/**
-	 * Exit run loop with given error and release all resources.
-	 *
-	 * @param t Error cause
-	 */
-	void stopWithError(Throwable t);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java
new file mode 100644
index 0000000..91fdc71
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java
@@ -0,0 +1,446 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import kafka.common.TopicAndPartition;
+import org.apache.kafka.common.Node;
+
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.SerializedValue;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A fetcher that fetches data from Kafka brokers via the Kafka 0.8 low-level consumer API.
+ * The fetcher also handles the explicit communication with ZooKeeper to fetch initial offsets
+ * and to write offsets to ZooKeeper.
+ *
+ * @param <T> The type of elements produced by the fetcher.
+ */
+public class Kafka08Fetcher<T> extends AbstractFetcher<T, TopicAndPartition> {
+	
+	static final KafkaTopicPartitionState<TopicAndPartition> MARKER = 
+			new KafkaTopicPartitionState<>(new KafkaTopicPartition("n/a", -1), new TopicAndPartition("n/a", -1));
+
+	private static final Logger LOG = LoggerFactory.getLogger(Kafka08Fetcher.class);
+
+	// ------------------------------------------------------------------------
+
+	/** The schema to convert between Kafka's byte messages, and Flink's objects */
+	private final KeyedDeserializationSchema<T> deserializer;
+
+	/** The properties that configure the Kafka connection */
+	private final Properties kafkaConfig;
+
+	/** The task name, to give more readable names to the spawned threads */
+	private final String taskName;
+
+	/** The class loader for dynamically loaded classes */
+	private final ClassLoader userCodeClassLoader;
+
+	/** The queue of partitions that are currently not assigned to a broker connection */
+	private final ClosableBlockingQueue<KafkaTopicPartitionState<TopicAndPartition>> unassignedPartitionsQueue;
+
+	/** The behavior to use in case that an offset is not valid (any more) for a partition */
+	private final long invalidOffsetBehavior;
+
+	/** The interval in which to automatically commit (-1 if deactivated) */
+	private final long autoCommitInterval; 
+
+	/** The handler that reads/writes offsets from/to ZooKeeper */
+	private volatile ZookeeperOffsetHandler zookeeperOffsetHandler;
+
+	/** Flag to track the main work loop as alive */
+	private volatile boolean running = true;
+
+
+	public Kafka08Fetcher(
+			SourceContext<T> sourceContext,
+			List<KafkaTopicPartition> assignedPartitions,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			StreamingRuntimeContext runtimeContext,
+			KeyedDeserializationSchema<T> deserializer,
+			Properties kafkaProperties,
+			long invalidOffsetBehavior,
+			long autoCommitInterval) throws Exception
+	{
+		super(sourceContext, assignedPartitions, watermarksPeriodic, watermarksPunctuated, runtimeContext);
+
+		this.deserializer = checkNotNull(deserializer);
+		this.kafkaConfig = checkNotNull(kafkaProperties);
+		this.taskName = runtimeContext.getTaskNameWithSubtasks();
+		this.userCodeClassLoader = runtimeContext.getUserCodeClassLoader();
+		this.invalidOffsetBehavior = invalidOffsetBehavior;
+		this.autoCommitInterval = autoCommitInterval;
+		this.unassignedPartitionsQueue = new ClosableBlockingQueue<>();
+
+		// initially, all these partitions are not assigned to a specific broker connection
+		for (KafkaTopicPartitionState<TopicAndPartition> partition : subscribedPartitions()) {
+			unassignedPartitionsQueue.add(partition);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Main Work Loop
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void runFetchLoop() throws Exception {
+		// the map from broker to the thread that is connected to that broker
+		final Map<Node, SimpleConsumerThread<T>> brokerToThread = new HashMap<>();
+
+		// this holds possible the exceptions from the concurrent broker connection threads
+		final ExceptionProxy errorHandler = new ExceptionProxy(Thread.currentThread());
+
+		// the offset handler handles the communication with ZooKeeper, to commit externally visible offsets
+		final ZookeeperOffsetHandler zookeeperOffsetHandler = new ZookeeperOffsetHandler(kafkaConfig);
+		this.zookeeperOffsetHandler = zookeeperOffsetHandler;
+
+		PeriodicOffsetCommitter periodicCommitter = null;
+		try {
+			// read offsets from ZooKeeper for partitions that did not restore offsets
+			{
+				List<KafkaTopicPartition> partitionsWithNoOffset = new ArrayList<>();
+				for (KafkaTopicPartitionState<TopicAndPartition> partition : subscribedPartitions()) {
+					if (!partition.isOffsetDefined()) {
+						partitionsWithNoOffset.add(partition.getKafkaTopicPartition());
+					}
+				}
+
+				Map<KafkaTopicPartition, Long> zkOffsets = zookeeperOffsetHandler.getOffsets(partitionsWithNoOffset);
+				for (KafkaTopicPartitionState<TopicAndPartition> partition : subscribedPartitions()) {
+					Long offset = zkOffsets.get(partition.getKafkaTopicPartition());
+					if (offset != null) {
+						partition.setOffset(offset);
+					}
+				}
+			}
+
+			// start the periodic offset committer thread, if necessary
+			if (autoCommitInterval > 0) {
+				LOG.info("Starting periodic offset committer, with commit interval of {}ms", autoCommitInterval);
+
+				periodicCommitter = new PeriodicOffsetCommitter(zookeeperOffsetHandler, 
+						subscribedPartitions(), errorHandler, autoCommitInterval);
+				periodicCommitter.setName("Periodic Kafka partition offset committer");
+				periodicCommitter.setDaemon(true);
+				periodicCommitter.start();
+			}
+
+			// Main loop polling elements from the unassignedPartitions queue to the threads
+			while (running) {
+				// re-throw any exception from the concurrent fetcher threads
+				errorHandler.checkAndThrowException();
+
+				// wait for max 5 seconds trying to get partitions to assign
+				// if threads shut down, this poll returns earlier, because the threads inject the
+				// special marker into the queue
+				List<KafkaTopicPartitionState<TopicAndPartition>> partitionsToAssign = 
+						unassignedPartitionsQueue.getBatchBlocking(5000);
+				partitionsToAssign.remove(MARKER);
+
+				if (!partitionsToAssign.isEmpty()) {
+					LOG.info("Assigning {} partitions to broker threads", partitionsToAssign.size());
+					Map<Node, List<KafkaTopicPartitionState<TopicAndPartition>>> partitionsWithLeaders = 
+							findLeaderForPartitions(partitionsToAssign, kafkaConfig);
+
+					// assign the partitions to the leaders (maybe start the threads)
+					for (Map.Entry<Node, List<KafkaTopicPartitionState<TopicAndPartition>>> partitionsWithLeader : 
+							partitionsWithLeaders.entrySet())
+					{
+						final Node leader = partitionsWithLeader.getKey();
+						final List<KafkaTopicPartitionState<TopicAndPartition>> partitions = partitionsWithLeader.getValue();
+						SimpleConsumerThread<T> brokerThread = brokerToThread.get(leader);
+
+						if (!running) {
+							break;
+						}
+
+						if (brokerThread == null || !brokerThread.getNewPartitionsQueue().isOpen()) {
+							// start new thread
+							brokerThread = createAndStartSimpleConsumerThread(partitions, leader, errorHandler);
+							brokerToThread.put(leader, brokerThread);
+						}
+						else {
+							// put elements into queue of thread
+							ClosableBlockingQueue<KafkaTopicPartitionState<TopicAndPartition>> newPartitionsQueue = 
+									brokerThread.getNewPartitionsQueue();
+							
+							for (KafkaTopicPartitionState<TopicAndPartition> fp : partitions) {
+								if (!newPartitionsQueue.addIfOpen(fp)) {
+									// we were unable to add the partition to the broker's queue
+									// the broker has closed in the meantime (the thread will shut down)
+									// create a new thread for connecting to this broker
+									List<KafkaTopicPartitionState<TopicAndPartition>> seedPartitions = new ArrayList<>();
+									seedPartitions.add(fp);
+									brokerThread = createAndStartSimpleConsumerThread(seedPartitions, leader, errorHandler);
+									brokerToThread.put(leader, brokerThread);
+									newPartitionsQueue = brokerThread.getNewPartitionsQueue(); // update queue for the subsequent partitions
+								}
+							}
+						}
+					}
+				}
+				else {
+					// there were no partitions to assign. Check if any broker threads shut down.
+					// we get into this section of the code, if either the poll timed out, or the
+					// blocking poll was woken up by the marker element
+					Iterator<SimpleConsumerThread<T>> bttIterator = brokerToThread.values().iterator();
+					while (bttIterator.hasNext()) {
+						SimpleConsumerThread<T> thread = bttIterator.next();
+						if (!thread.getNewPartitionsQueue().isOpen()) {
+							LOG.info("Removing stopped consumer thread {}", thread.getName());
+							bttIterator.remove();
+						}
+					}
+				}
+
+				if (brokerToThread.size() == 0 && unassignedPartitionsQueue.isEmpty()) {
+					if (unassignedPartitionsQueue.close()) {
+						LOG.info("All consumer threads are finished, there are no more unassigned partitions. Stopping fetcher");
+						break;
+					}
+					// we end up here if somebody added something to the queue in the meantime --> continue to poll queue again
+				}
+			}
+		}
+		catch (InterruptedException e) {
+			// this may be thrown because an exception on one of the concurrent fetcher threads
+			// woke this thread up. make sure we throw the root exception instead in that case
+			errorHandler.checkAndThrowException();
+
+			// no other root exception, throw the interrupted exception
+			throw e;
+		}
+		finally {
+			this.running = false;
+			this.zookeeperOffsetHandler = null;
+
+			// if we run a periodic committer thread, shut that down
+			if (periodicCommitter != null) {
+				periodicCommitter.shutdown();
+			}
+
+			// make sure that in any case (completion, abort, error), all spawned threads are stopped
+			try {
+				int runningThreads;
+				do {
+					// check whether threads are alive and cancel them
+					runningThreads = 0;
+					Iterator<SimpleConsumerThread<T>> threads = brokerToThread.values().iterator();
+					while (threads.hasNext()) {
+						SimpleConsumerThread<?> t = threads.next();
+						if (t.isAlive()) {
+							t.cancel();
+							runningThreads++;
+						} else {
+							threads.remove();
+						}
+					}
+
+					// wait for the threads to finish, before issuing a cancel call again
+					if (runningThreads > 0) {
+						for (SimpleConsumerThread<?> t : brokerToThread.values()) {
+							t.join(500 / runningThreads + 1);
+						}
+					}
+				}
+				while (runningThreads > 0);
+			}
+			catch (Throwable t) {
+				// we catch all here to preserve the original exception
+				LOG.error("Exception while shutting down consumer threads", t);
+			}
+
+			try {
+				zookeeperOffsetHandler.close();
+			}
+			catch (Throwable t) {
+				// we catch all here to preserve the original exception
+				LOG.error("Exception while shutting down ZookeeperOffsetHandler", t);
+			}
+		}
+	}
+
+	@Override
+	public void cancel() {
+		// signal the main thread to exit
+		this.running = false;
+
+		// make sure the main thread wakes up soon
+		this.unassignedPartitionsQueue.addIfOpen(MARKER);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Kafka 0.8 specific class instantiation
+	// ------------------------------------------------------------------------
+
+	@Override
+	public TopicAndPartition createKafkaPartitionHandle(KafkaTopicPartition partition) {
+		return new TopicAndPartition(partition.getTopic(), partition.getPartition());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Offset handling
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void commitSpecificOffsetsToKafka(Map<KafkaTopicPartition, Long> offsets) throws Exception {
+		ZookeeperOffsetHandler zkHandler = this.zookeeperOffsetHandler;
+		if (zkHandler != null) {
+			zkHandler.writeOffsets(offsets);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private SimpleConsumerThread<T> createAndStartSimpleConsumerThread(
+			List<KafkaTopicPartitionState<TopicAndPartition>> seedPartitions,
+			Node leader,
+			ExceptionProxy errorHandler) throws IOException, ClassNotFoundException
+	{
+		// each thread needs its own copy of the deserializer, because the deserializer is
+		// not necessarily thread safe
+		final KeyedDeserializationSchema<T> clonedDeserializer =
+				InstantiationUtil.clone(deserializer, userCodeClassLoader);
+
+		// seed thread with list of fetch partitions (otherwise it would shut down immediately again
+		SimpleConsumerThread<T> brokerThread = new SimpleConsumerThread<>(
+				this, errorHandler, kafkaConfig, leader, seedPartitions, unassignedPartitionsQueue, 
+				clonedDeserializer, invalidOffsetBehavior);
+
+		brokerThread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)",
+				taskName, leader.id(), leader.host(), leader.port()));
+		brokerThread.setDaemon(true);
+		brokerThread.start();
+
+		LOG.info("Starting thread {}", brokerThread.getName());
+		return brokerThread;
+	}
+
+	/**
+	 * Returns a list of unique topics from for the given partitions
+	 *
+	 * @param partitions A the partitions
+	 * @return A list of unique topics
+	 */
+	private static List<String> getTopics(List<KafkaTopicPartitionState<TopicAndPartition>> partitions) {
+		HashSet<String> uniqueTopics = new HashSet<>();
+		for (KafkaTopicPartitionState<TopicAndPartition> fp: partitions) {
+			uniqueTopics.add(fp.getTopic());
+		}
+		return new ArrayList<>(uniqueTopics);
+	}
+
+	/**
+	 * Find leaders for the partitions
+	 *
+	 * From a high level, the method does the following:
+	 *	 - Get a list of FetchPartitions (usually only a few partitions)
+	 *	 - Get the list of topics from the FetchPartitions list and request the partitions for the topics. (Kafka doesn't support getting leaders for a set of partitions)
+	 *	 - Build a Map<Leader, List<FetchPartition>> where only the requested partitions are contained.
+	 *
+	 * @param partitionsToAssign fetch partitions list
+	 * @return leader to partitions map
+	 */
+	private static Map<Node, List<KafkaTopicPartitionState<TopicAndPartition>>> findLeaderForPartitions(
+			List<KafkaTopicPartitionState<TopicAndPartition>> partitionsToAssign,
+			Properties kafkaProperties) throws Exception
+	{
+		if (partitionsToAssign.isEmpty()) {
+			throw new IllegalArgumentException("Leader request for empty partitions list");
+		}
+
+		LOG.info("Refreshing leader information for partitions {}", partitionsToAssign);
+		
+		// this request is based on the topic names
+		PartitionInfoFetcher infoFetcher = new PartitionInfoFetcher(getTopics(partitionsToAssign), kafkaProperties);
+		infoFetcher.start();
+
+		// NOTE: The kafka client apparently locks itself up sometimes
+		// when it is interrupted, so we run it only in a separate thread.
+		// since it sometimes refuses to shut down, we resort to the admittedly harsh
+		// means of killing the thread after a timeout.
+		KillerWatchDog watchDog = new KillerWatchDog(infoFetcher, 60000);
+		watchDog.start();
+
+		// this list contains ALL partitions of the requested topics
+		List<KafkaTopicPartitionLeader> topicPartitionWithLeaderList = infoFetcher.getPartitions();
+
+		// copy list to track unassigned partitions
+		List<KafkaTopicPartitionState<TopicAndPartition>> unassignedPartitions = new ArrayList<>(partitionsToAssign);
+
+		// final mapping from leader -> list(fetchPartition)
+		Map<Node, List<KafkaTopicPartitionState<TopicAndPartition>>> leaderToPartitions = new HashMap<>();
+
+		for(KafkaTopicPartitionLeader partitionLeader: topicPartitionWithLeaderList) {
+			if (unassignedPartitions.size() == 0) {
+				// we are done: all partitions are assigned
+				break;
+			}
+
+			Iterator<KafkaTopicPartitionState<TopicAndPartition>> unassignedPartitionsIterator = unassignedPartitions.iterator();
+			while (unassignedPartitionsIterator.hasNext()) {
+				KafkaTopicPartitionState<TopicAndPartition> unassignedPartition = unassignedPartitionsIterator.next();
+
+				if (unassignedPartition.getKafkaTopicPartition().equals(partitionLeader.getTopicPartition())) {
+					// we found the leader for one of the fetch partitions
+					Node leader = partitionLeader.getLeader();
+
+					List<KafkaTopicPartitionState<TopicAndPartition>> partitionsOfLeader = leaderToPartitions.get(leader);
+					if (partitionsOfLeader == null) {
+						partitionsOfLeader = new ArrayList<>();
+						leaderToPartitions.put(leader, partitionsOfLeader);
+					}
+					partitionsOfLeader.add(unassignedPartition);
+					unassignedPartitionsIterator.remove(); // partition has been assigned
+					break;
+				}
+			}
+		}
+
+		if (unassignedPartitions.size() > 0) {
+			throw new RuntimeException("Unable to find a leader for partitions: " + unassignedPartitions);
+		}
+
+		LOG.debug("Partitions with assigned leaders {}", leaderToPartitions);
+
+		return leaderToPartitions;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java
new file mode 100644
index 0000000..4d61e53
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KillerWatchDog.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+/**
+ * A watch dog thread that forcibly kills another thread, if that thread does not
+ * finish in time.
+ * 
+ * <p>This uses the discouraged {@link Thread#stop()} method. While this is not
+ * advisable, this watch dog is only for extreme cases of thread that simply
+ * to not terminate otherwise.
+ */
+class KillerWatchDog extends Thread {
+
+	private final Thread toKill;
+	private final long timeout;
+
+	KillerWatchDog(Thread toKill, long timeout) {
+		super("KillerWatchDog");
+		setDaemon(true);
+
+		this.toKill = toKill;
+		this.timeout = timeout;
+	}
+
+	@SuppressWarnings("deprecation")
+	@Override
+	public void run() {
+		final long deadline = System.currentTimeMillis() + timeout;
+		long now;
+
+		while (toKill.isAlive() && (now = System.currentTimeMillis()) < deadline) {
+			try {
+				toKill.join(deadline - now);
+			}
+			catch (InterruptedException e) {
+				// ignore here, our job is important!
+			}
+		}
+
+		// this is harsh, but this watchdog is a last resort
+		if (toKill.isAlive()) {
+			toKill.stop();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
deleted file mode 100644
index c4dd55c..0000000
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
+++ /dev/null
@@ -1,896 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.internals;
-
-import kafka.api.FetchRequestBuilder;
-import kafka.api.OffsetRequest;
-import kafka.api.PartitionOffsetRequestInfo;
-import kafka.common.ErrorMapping;
-import kafka.common.TopicAndPartition;
-import kafka.javaapi.FetchResponse;
-import kafka.javaapi.OffsetResponse;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.MessageAndOffset;
-
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.util.InstantiationUtil;
-import org.apache.flink.util.StringUtils;
-
-import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.common.Node;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Objects.requireNonNull;
-import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.getIntFromConfig;
-
-/**
- * This fetcher uses Kafka's low-level API to pull data from a specific
- * set of topics and partitions.
- * 
- * <p>This code is in parts based on the tutorial code for the low-level Kafka consumer.</p>
- */
-public class LegacyFetcher<T> implements Fetcher {
-	
-	private static final Logger LOG = LoggerFactory.getLogger(LegacyFetcher.class);
-
-	private static final FetchPartition MARKER = new FetchPartition("n/a", -1, -1);
-
-	/** The properties that configure the Kafka connection */
-	private final Properties config;
-	
-	/** The task name, to give more readable names to the spawned threads */
-	private final String taskName;
-	
-	/** The first error that occurred in a connection thread */
-	private final AtomicReference<Throwable> error;
-	
-	/** The classloader for dynamically loaded classes */
-	private final ClassLoader userCodeClassloader;
-	
-	/** Reference the the thread that executed the run() method. */
-	private volatile Thread mainThread;
-	
-	/** Flag to shut the fetcher down */
-	private volatile boolean running = true;
-
-	/**
-	 * Queue of partitions which need to find a (new) leader. Elements are added to the queue
-	 * from the consuming threads. The LegacyFetcher thread is finding new leaders and assigns the partitions
-	 * to the respective threads.
-	 */
-	private final ClosableBlockingQueue<FetchPartition> unassignedPartitions = new ClosableBlockingQueue<>();
-
-	/** The {@link FlinkKafkaConsumer08} to whom this Fetcher belongs. */
-	private final FlinkKafkaConsumer08<T> flinkKafkaConsumer;
-
-	/**
-	 * Create a LegacyFetcher instance.
-	 *
-	 * @param initialPartitionsToRead Map of partitions to read. The offset passed is the last-fetched-offset (not the next-offset-to-fetch).
-	 * @param props kafka properties
-	 * @param taskName name of the parent task
-	 * @param userCodeClassloader classloader for loading user code
-	 */
-	public LegacyFetcher(
-				FlinkKafkaConsumer08<T> owner,
-				Map<KafkaTopicPartition, KafkaPartitionState> initialPartitionsToRead,
-				Properties props,
-				String taskName, ClassLoader userCodeClassloader) {
-
-		this.flinkKafkaConsumer = requireNonNull(owner);
-		this.config = requireNonNull(props, "The config properties cannot be null");
-		this.userCodeClassloader = requireNonNull(userCodeClassloader);
-		if (initialPartitionsToRead.size() == 0) {
-			throw new IllegalArgumentException("List of initial partitions is empty");
-		}
-
-		for (Map.Entry<KafkaTopicPartition, KafkaPartitionState> partitionToRead: initialPartitionsToRead.entrySet()) {
-			KafkaTopicPartition ktp = partitionToRead.getKey();
-			// we increment the offset by one so that we fetch the next message in the partition.
-			long offset = partitionToRead.getValue().getOffset();
-			if (offset >= 0 && offset != FlinkKafkaConsumerBase.OFFSET_NOT_SET) {
-				offset += 1L;
-			}
-			unassignedPartitions.add(new FetchPartition(ktp.getTopic(), ktp.getPartition(), offset));
-		}
-		this.taskName = taskName;
-		this.error = new AtomicReference<>();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Fetcher methods
-	// ------------------------------------------------------------------------
-
-
-	@Override
-	public void close() {
-		// flag needs to be check by the run() method that creates the spawned threads
-		this.running = false;
-		
-		// all other cleanup is made by the run method itself
-	}
-
-	@Override
-	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
-						KeyedDeserializationSchema<T> deserializer,
-						HashMap<KafkaTopicPartition, KafkaPartitionState> partitionState) throws Exception {
-
-		// NOTE: This method needs to always release all resources it acquires
-
-		this.mainThread = Thread.currentThread();
-
-		// keep presumably dead threads in the list until we are sure the thread is not alive anymore.
-		final Map<Node, SimpleConsumerThread<T>> brokerToThread = new HashMap<>();
-		try {
-			// Main loop polling elements from the unassignedPartitions queue to the threads
-			while (running && error.get() == null) {
-				try {
-					// wait for 5 seconds trying to get partitions to assign
-					List<FetchPartition> partitionsToAssign = unassignedPartitions.getBatchBlocking(5000);
-					partitionsToAssign.remove(MARKER);
-					
-					if(!partitionsToAssign.isEmpty()) {
-						LOG.info("Assigning {} partitions to broker threads", partitionsToAssign.size());
-						Map<Node, List<FetchPartition>> partitionsWithLeaders = findLeaderForPartitions(partitionsToAssign);
-						
-						// assign the partitions to the leaders (maybe start the threads)
-						for (Map.Entry<Node, List<FetchPartition>> partitionsWithLeader : partitionsWithLeaders.entrySet()) {
-							final Node leader = partitionsWithLeader.getKey();
-							final List<FetchPartition> partitions = partitionsWithLeader.getValue();
-							SimpleConsumerThread<T> brokerThread = brokerToThread.get(leader);
-
-							if (!running) {
-								break;
-							}
-							
-							if (brokerThread == null || !brokerThread.getNewPartitionsQueue().isOpen()) {
-								// start new thread
-								brokerThread = createAndStartSimpleConsumerThread(sourceContext, deserializer, partitions, leader);
-								brokerToThread.put(leader, brokerThread);
-							} else {
-								// put elements into queue of thread
-								ClosableBlockingQueue<FetchPartition> newPartitionsQueue = brokerThread.getNewPartitionsQueue();
-								for (FetchPartition fp : partitions) {
-									if (!newPartitionsQueue.addIfOpen(fp)) {
-										// we were unable to add the partition to the broker's queue
-										// the broker has closed in the meantime (the thread will shut down)
-										// create a new thread for connecting to this broker
-										List<FetchPartition> seedPartitions = new ArrayList<>();
-										seedPartitions.add(fp);
-										brokerThread = createAndStartSimpleConsumerThread(sourceContext, deserializer, seedPartitions, leader);
-										brokerToThread.put(leader, brokerThread);
-										newPartitionsQueue = brokerThread.getNewPartitionsQueue(); // update queue for the subsequent partitions
-									}
-								}
-							}
-						}
-					} else {
-						// there were no partitions to assign. Check if any broker threads shut down.
-						Iterator<SimpleConsumerThread<T>> bttIterator = brokerToThread.values().iterator();
-						while (bttIterator.hasNext()) {
-							SimpleConsumerThread<T> thread = bttIterator.next();
-							if(!thread.getNewPartitionsQueue().isOpen()) {
-								LOG.info("Removing stopped consumer thread {}", thread.getName());
-								bttIterator.remove();
-							}
-						}
-					}
-
-					if (brokerToThread.size() == 0 && unassignedPartitions.isEmpty()) {
-						if (unassignedPartitions.close()) {
-							LOG.info("All consumer threads are finished, there are no more unassigned partitions. Stopping fetcher");
-							break;
-						}
-						// we end up here if somebody added something to the queue in the meantime --> continue to poll queue again
-					}
-				} catch (InterruptedException e) {
-					// ignore. we should notice what happened in the next loop check
-				}
-			}
-
-			// make sure any asynchronous error is noticed
-			Throwable error = this.error.get();
-			if (error != null) {
-				throw new Exception(error.getMessage(), error);
-			}
-		} finally {
-			// make sure that in any case (completion, abort, error), all spawned threads are stopped
-			int runningThreads;
-			do {
-				runningThreads = 0;
-				for (SimpleConsumerThread<?> t : brokerToThread.values()) {
-					if (t.isAlive()) {
-						t.cancel();
-						runningThreads++;
-					}
-				}
-				if(runningThreads > 0) {
-					Thread.sleep(500);
-				}
-			} while(runningThreads > 0);
-		}
-	}
-
-	private <T> SimpleConsumerThread<T> createAndStartSimpleConsumerThread(SourceFunction.SourceContext<T> sourceContext,
-																		KeyedDeserializationSchema<T> deserializer,
-																		List<FetchPartition> seedPartitions, Node leader) throws IOException, ClassNotFoundException {
-		final KeyedDeserializationSchema<T> clonedDeserializer =
-				InstantiationUtil.clone(deserializer, userCodeClassloader);
-
-		// seed thread with list of fetch partitions (otherwise it would shut down immediately again
-		SimpleConsumerThread<T> brokerThread = new SimpleConsumerThread<>(this, config,
-				leader, seedPartitions, unassignedPartitions, sourceContext, clonedDeserializer);
-
-		brokerThread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)",
-				taskName, leader.id(), leader.host(), leader.port()));
-		brokerThread.setDaemon(true);
-		brokerThread.start();
-		LOG.info("Starting thread {}", brokerThread.getName());
-		return brokerThread;
-	}
-
-	/**
-	 * Find leaders for the partitions
-	 *
-	 * From a high level, the method does the following:
-	 *	 - Get a list of FetchPartitions (usually only a few partitions)
-	 *	 - Get the list of topics from the FetchPartitions list and request the partitions for the topics. (Kafka doesn't support getting leaders for a set of partitions)
-	 *	 - Build a Map<Leader, List<FetchPartition>> where only the requested partitions are contained.
-	 *
-	 * @param partitionsToAssign fetch partitions list
-	 * @return leader to partitions map
-	 */
-	private Map<Node, List<FetchPartition>> findLeaderForPartitions(List<FetchPartition> partitionsToAssign) throws Exception {
-		if(partitionsToAssign.size() == 0) {
-			throw new IllegalArgumentException("Leader request for empty partitions list");
-		}
-
-		LOG.info("Refreshing leader information for partitions {}", partitionsToAssign);
-		// NOTE: The kafka client apparently locks itself in an infinite loop sometimes
-		// when it is interrupted, so we run it only in a separate thread.
-		// since it sometimes refuses to shut down, we resort to the admittedly harsh
-		// means of killing the thread after a timeout.
-		PartitionInfoFetcher infoFetcher = new PartitionInfoFetcher(getTopics(partitionsToAssign), config); // this request is based on the topic names
-		infoFetcher.start();
-
-		KillerWatchDog watchDog = new KillerWatchDog(infoFetcher, 60000);
-		watchDog.start();
-
-		// this list contains ALL partitions of the requested topics
-		List<KafkaTopicPartitionLeader> topicPartitionWithLeaderList = infoFetcher.getPartitions();
-		// copy list to track unassigned partitions
-		List<FetchPartition> unassignedPartitions = new ArrayList<>(partitionsToAssign);
-
-		// final mapping from leader -> list(fetchPartition)
-		Map<Node, List<FetchPartition>> leaderToPartitions = new HashMap<>();
-
-		for(KafkaTopicPartitionLeader partitionLeader: topicPartitionWithLeaderList) {
-			if (unassignedPartitions.size() == 0) {
-				// we are done: all partitions are assigned
-				break;
-			}
-			Iterator<FetchPartition> unassignedPartitionsIterator = unassignedPartitions.iterator();
-			while (unassignedPartitionsIterator.hasNext()) {
-				FetchPartition unassignedPartition = unassignedPartitionsIterator.next();
-				if (unassignedPartition.topic.equals(partitionLeader.getTopicPartition().getTopic())
-						&& unassignedPartition.partition == partitionLeader.getTopicPartition().getPartition()) {
-
-					// we found the leader for one of the fetch partitions
-					Node leader = partitionLeader.getLeader();
-					List<FetchPartition> partitionsOfLeader = leaderToPartitions.get(leader);
-					if (partitionsOfLeader == null) {
-						partitionsOfLeader = new ArrayList<>();
-						leaderToPartitions.put(leader, partitionsOfLeader);
-					}
-					partitionsOfLeader.add(unassignedPartition);
-					unassignedPartitionsIterator.remove(); // partition has been assigned
-					break;
-				}
-			}
-		}
-
-		if(unassignedPartitions.size() > 0) {
-			throw new RuntimeException("Unable to find a leader for partitions: " + unassignedPartitions);
-		}
-
-		LOG.debug("Partitions with assigned leaders {}", leaderToPartitions);
-
-		return leaderToPartitions;
-	}
-
-	/**
-	 * Reports an error from a fetch thread. This will cause the main thread to see this error,
-	 * abort, and cancel all other fetch threads.
-	 * 
-	 * @param error The error to report.
-	 */
-	@Override
-	public void stopWithError(Throwable error) {
-		if (this.error.compareAndSet(null, error)) {
-			// we are the first to report an error
-			if (mainThread != null) {
-				mainThread.interrupt();
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Representation of a partition to fetch.
-	 */
-	private static class FetchPartition {
-
-		final String topic;
-		
-		/** ID of the partition within the topic (0 indexed, as given by Kafka) */
-		final int partition;
-		
-		/** Offset pointing at the next element to read from that partition. */
-		long nextOffsetToRead;
-
-		FetchPartition(String topic, int partition, long nextOffsetToRead) {
-			this.topic = topic;
-			this.partition = partition;
-			this.nextOffsetToRead = nextOffsetToRead;
-		}
-		
-		@Override
-		public String toString() {
-			return "FetchPartition {topic=" + topic +", partition=" + partition + ", offset=" + nextOffsetToRead + '}';
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Per broker fetcher
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Each broker needs its separate connection. This thread implements the connection to
-	 * one broker. The connection can fetch multiple partitions from the broker.
-	 * 
-	 * @param <T> The data type fetched.
-	 */
-	private static class SimpleConsumerThread<T> extends Thread {
-
-		private static final Logger LOG = LoggerFactory.getLogger(SimpleConsumerThread.class);
-		
-		private final SourceFunction.SourceContext<T> sourceContext;
-		private final KeyedDeserializationSchema<T> deserializer;
-
-		private final List<FetchPartition> partitions;
-		
-		private final Node broker;
-
-		private final Properties config;
-
-		private final LegacyFetcher owner;
-
-		private final ClosableBlockingQueue<FetchPartition> unassignedPartitions;
-
-		private volatile boolean running = true;
-
-		/** Queue containing new fetch partitions for the consumer thread */
-		private final ClosableBlockingQueue<FetchPartition> newPartitionsQueue = new ClosableBlockingQueue<>();
-
-		// ----------------- Simple Consumer ----------------------
-		private SimpleConsumer consumer;
-
-		private final int soTimeout;
-		private final int minBytes;
-		private final int maxWait;
-		private final int fetchSize;
-		private final int bufferSize;
-		private final int reconnectLimit;
-
-
-		// exceptions are thrown locally
-		public SimpleConsumerThread(LegacyFetcher owner,
-									Properties config,
-									Node broker,
-									List<FetchPartition> seedPartitions,
-									ClosableBlockingQueue<FetchPartition> unassignedPartitions,
-									SourceFunction.SourceContext<T> sourceContext,
-									KeyedDeserializationSchema<T> deserializer) {
-			this.owner = owner;
-			this.config = config;
-			this.broker = broker;
-			this.partitions = seedPartitions;
-			this.sourceContext = requireNonNull(sourceContext);
-			this.deserializer = requireNonNull(deserializer);
-			this.unassignedPartitions = requireNonNull(unassignedPartitions);
-
-			this.soTimeout = getIntFromConfig(config, "socket.timeout.ms", 30000);
-			this.minBytes = getIntFromConfig(config, "fetch.min.bytes", 1);
-			this.maxWait = getIntFromConfig(config, "fetch.wait.max.ms", 100);
-			this.fetchSize = getIntFromConfig(config, "fetch.message.max.bytes", 1048576);
-			this.bufferSize = getIntFromConfig(config, "socket.receive.buffer.bytes", 65536);
-			this.reconnectLimit = getIntFromConfig(config, "flink.simple-consumer-reconnectLimit", 3);
-		}
-
-		@Override
-		public void run() {
-			LOG.info("Starting to fetch from {}", this.partitions);
-
-			// set up the config values
-			final String clientId = "flink-kafka-consumer-legacy-" + broker.id();
-			int reconnects = 0;
-			// these are the actual configuration values of Kafka + their original default values.
-
-			try {
-				// create the Kafka consumer that we actually use for fetching
-				consumer = new SimpleConsumer(broker.host(), broker.port(), soTimeout, bufferSize, clientId);
-
-				// make sure that all partitions have some offsets to start with
-				// those partitions that do not have an offset from a checkpoint need to get
-				// their start offset from ZooKeeper
-				getMissingOffsetsFromKafka(partitions);
-
-				// Now, the actual work starts :-)
-				int offsetOutOfRangeCount = 0;
-				while (running) {
-
-					// ----------------------------------- partitions list maintenance ----------------------------
-
-					// check queue for new partitions to read from:
-					List<FetchPartition> newPartitions = newPartitionsQueue.pollBatch();
-					if(newPartitions != null) {
-						// only this thread is taking elements from the queue, so the next call will never block
-
-						// check if the new partitions need an offset lookup
-						getMissingOffsetsFromKafka(newPartitions);
-						// add the new partitions (and check they are not already in there)
-						for(FetchPartition newPartition: newPartitions) {
-							if(partitions.contains(newPartition)) {
-								throw new IllegalStateException("Adding partition " + newPartition + " to subscribed partitions even though it is already subscribed");
-							}
-							partitions.add(newPartition);
-						}
-						LOG.info("Adding {} new partitions to consumer thread {}", newPartitions.size(), getName());
-						if(LOG.isDebugEnabled()) {
-							LOG.debug("Partitions list: {}", newPartitions);
-						}
-					}
-
-					if(partitions.size() == 0) {
-						if(newPartitionsQueue.close()) {
-							// close succeeded. Closing thread
-							LOG.info("Consumer thread {} does not have any partitions assigned anymore. Stopping thread.", getName());
-							running = false;
-							
-							// add the wake-up marker into the queue to make the main thread
-							// immediately wake up and termination faster
-							unassignedPartitions.add(MARKER);
-							
-							break;
-						} else {
-							// close failed: LegacyFetcher main thread added new partitions into the queue.
-							continue; // go to top of loop again and get the new partitions
-						}
-					}
-
-					// ----------------------------------- request / response with kafka ----------------------------
-
-					FetchRequestBuilder frb = new FetchRequestBuilder();
-					frb.clientId(clientId);
-					frb.maxWait(maxWait);
-					frb.minBytes(minBytes);
-					
-					for (FetchPartition fp : partitions) {
-						frb.addFetch(fp.topic, fp.partition, fp.nextOffsetToRead, fetchSize);
-					}
-					kafka.api.FetchRequest fetchRequest = frb.build();
-					LOG.debug("Issuing fetch request {}", fetchRequest);
-
-					FetchResponse fetchResponse;
-					try {
-						fetchResponse = consumer.fetch(fetchRequest);
-					} catch(Throwable cce) {
-						//noinspection ConstantConditions
-						if(cce instanceof ClosedChannelException) {
-							LOG.warn("Fetch failed because of ClosedChannelException.");
-							LOG.debug("Full exception", cce);
-							// we don't know if the broker is overloaded or unavailable.
-							// retry a few times, then return ALL partitions for new leader lookup
-							if(++reconnects >= reconnectLimit) {
-								LOG.warn("Unable to reach broker after {} retries. Returning all current partitions", reconnectLimit);
-								for (FetchPartition fp: this.partitions) {
-									unassignedPartitions.add(fp);
-								}
-								this.partitions.clear();
-								continue; // jump to top of loop: will close thread or subscribe to new partitions
-							}
-							try {
-								consumer.close();
-							} catch(Throwable t) {
-								LOG.warn("Error while closing consumer connection", t);
-							}
-							// delay & retry
-							Thread.sleep(500);
-							consumer = new SimpleConsumer(broker.host(), broker.port(), soTimeout, bufferSize, clientId);
-							continue; // retry
-						} else {
-							throw cce;
-						}
-					}
-					reconnects = 0;
-
-					// ---------------------------------------- error handling ----------------------------
-
-					if(fetchResponse == null) {
-						throw new RuntimeException("Fetch failed");
-					}
-					if (fetchResponse.hasError()) {
-						String exception = "";
-						List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<>();
-						// iterate over partitions to get individual error codes
-						Iterator<FetchPartition> partitionsIterator = partitions.iterator();
-						boolean partitionsRemoved = false;
-						while(partitionsIterator.hasNext()) {
-							final FetchPartition fp = partitionsIterator.next();
-							short code = fetchResponse.errorCode(fp.topic, fp.partition);
-
-							if (code == ErrorMapping.OffsetOutOfRangeCode()) {
-								// we were asked to read from an out-of-range-offset (maybe set wrong in Zookeeper)
-								// Kafka's high level consumer is resetting the offset according to 'auto.offset.reset'
-								partitionsToGetOffsetsFor.add(fp);
-							} else if(code == ErrorMapping.NotLeaderForPartitionCode() ||
-									code == ErrorMapping.LeaderNotAvailableCode() ||
-									code == ErrorMapping.BrokerNotAvailableCode() ||
-									code == ErrorMapping.UnknownCode()) {
-								// the broker we are connected to is not the leader for the partition.
-								LOG.warn("{} is not the leader of {}. Reassigning leader for partition", broker, fp);
-								LOG.debug("Error code = {}", code);
-								
-								unassignedPartitions.add(fp);
-								
-								partitionsIterator.remove(); // unsubscribe the partition ourselves
-								partitionsRemoved = true;
-							} else if (code != ErrorMapping.NoError()) {
-								exception += "\nException for " + fp.topic +":"+ fp.partition + ": " +
-										StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
-							}
-						}
-						if (partitionsToGetOffsetsFor.size() > 0) {
-							// safeguard against an infinite loop.
-							if (offsetOutOfRangeCount++ > 3) {
-								throw new RuntimeException("Found invalid offsets more than three times in partitions "+partitionsToGetOffsetsFor.toString()+" " +
-										"Exceptions: "+exception);
-							}
-							// get valid offsets for these partitions and try again.
-							LOG.warn("The following partitions had an invalid offset: {}", partitionsToGetOffsetsFor);
-							getLastOffset(consumer, partitionsToGetOffsetsFor, getInvalidOffsetBehavior(config));
-							LOG.warn("The new partition offsets are {}", partitionsToGetOffsetsFor);
-							continue; // jump back to create a new fetch request. The offset has not been touched.
-						} else if(partitionsRemoved) {
-							continue; // create new fetch request
-						} else {
-							// partitions failed on an error
-							throw new IOException("Error while fetching from broker '" + broker +"':" + exception);
-						}
-					} else {
-						// successful fetch, reset offsetOutOfRangeCount.
-						offsetOutOfRangeCount = 0;
-					}
-
-					// ----------------------------------- process fetch response ----------------------------
-
-					int messagesInFetch = 0;
-					int deletedMessages = 0;
-					Iterator<FetchPartition> partitionsIterator = partitions.iterator();
-					partitionsLoop: while (partitionsIterator.hasNext()) {
-						final FetchPartition fp = partitionsIterator.next();
-						final ByteBufferMessageSet messageSet = fetchResponse.messageSet(fp.topic, fp.partition);
-						final KafkaTopicPartition topicPartition = new KafkaTopicPartition(fp.topic, fp.partition);
-						
-						for (MessageAndOffset msg : messageSet) {
-							if (running) {
-								messagesInFetch++;
-								if (msg.offset() < fp.nextOffsetToRead) {
-									// we have seen this message already
-									LOG.info("Skipping message with offset " + msg.offset()
-											+ " because we have seen messages until " + fp.nextOffsetToRead
-											+ " from partition " + fp.partition + " already");
-									continue;
-								}
-
-								final long offset = msg.offset();
-
-								ByteBuffer payload = msg.message().payload();
-
-								// If the message value is null, this represents a delete command for the message key.
-								// Log this and pass it on to the client who might want to also receive delete messages.
-								byte[] valueBytes;
-								if (payload == null) {
-									deletedMessages++;
-									valueBytes = null;
-								} else {
-									valueBytes = new byte[payload.remaining()];
-									payload.get(valueBytes);
-								}
-
-								// put key into byte array
-								byte[] keyBytes = null;
-								int keySize = msg.message().keySize();
-
-								if (keySize >= 0) { // message().hasKey() is doing the same. We save one int deserialization
-									ByteBuffer keyPayload = msg.message().key();
-									keyBytes = new byte[keySize];
-									keyPayload.get(keyBytes);
-								}
-
-								final T value = deserializer.deserialize(keyBytes, valueBytes, fp.topic, fp.partition, offset);
-								if(deserializer.isEndOfStream(value)) {
-									// remove partition from subscribed partitions.
-									partitionsIterator.remove();
-									continue partitionsLoop;
-								}
-								synchronized (sourceContext.getCheckpointLock()) {
-									owner.flinkKafkaConsumer.processElement(sourceContext, topicPartition, value, offset);
-								}
-								
-								// advance offset for the next request
-								fp.nextOffsetToRead = offset + 1;
-							}
-							else {
-								// no longer running
-								return;
-							}
-						}
-					}
-					LOG.debug("This fetch contained {} messages ({} deleted messages)", messagesInFetch, deletedMessages);
-				} // end of fetch loop
-				
-				if (!newPartitionsQueue.close()) {
-					throw new Exception("Bug: Cleanly leaving fetcher thread without having a closed queue.");
-				}
-			}
-			catch (Throwable t) {
-				// report to the main thread
-				owner.stopWithError(t);
-			}
-			finally {
-				// end of run loop. close connection to consumer
-				if (consumer != null) {
-					// closing the consumer should not fail the program
-					try {
-						consumer.close();
-					}
-					catch (Throwable t) {
-						LOG.error("Error while closing the Kafka simple consumer", t);
-					}
-				}
-			}
-		}
-
-		private void getMissingOffsetsFromKafka(List<FetchPartition> partitions) {
-			List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<>();
-
-			for (FetchPartition fp : partitions) {
-				if (fp.nextOffsetToRead == FlinkKafkaConsumerBase.OFFSET_NOT_SET) {
-					// retrieve the offset from the consumer
-					partitionsToGetOffsetsFor.add(fp);
-				}
-			}
-			if (partitionsToGetOffsetsFor.size() > 0) {
-				getLastOffset(consumer, partitionsToGetOffsetsFor, getInvalidOffsetBehavior(config));
-				LOG.info("No prior offsets found for some partitions. Fetched the following start offsets {}", partitionsToGetOffsetsFor);
-
-				// setting the fetched offset also in the offset state.
-				// we subtract -1 from the offset
-				synchronized (sourceContext.getCheckpointLock()) {
-					for(FetchPartition fp: partitionsToGetOffsetsFor) {
-						owner.flinkKafkaConsumer.updateOffsetForPartition(new KafkaTopicPartition(fp.topic, fp.partition), fp.nextOffsetToRead - 1L);
-					}
-				}
-			}
-		}
-
-		/**
-		 * Cancels this fetch thread. The thread will release all resources and terminate.
-		 */
-		public void cancel() {
-			this.running = false;
-			
-			// interrupt whatever the consumer is doing
-			if (consumer != null) {
-				consumer.close();
-			}
-			
-			this.interrupt();
-		}
-
-		public ClosableBlockingQueue<FetchPartition> getNewPartitionsQueue() {
-			return newPartitionsQueue;
-		}
-
-		/**
-		 * Request latest offsets for a set of partitions, via a Kafka consumer.
-		 *
-		 * This method retries three times if the response has an error
-		 *
-		 * @param consumer The consumer connected to lead broker
-		 * @param partitions The list of partitions we need offsets for
-		 * @param whichTime The type of time we are requesting. -1 and -2 are special constants (See OffsetRequest)
-		 */
-		private static void getLastOffset(SimpleConsumer consumer, List<FetchPartition> partitions, long whichTime) {
-
-			Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<>();
-			for (FetchPartition fp: partitions) {
-				TopicAndPartition topicAndPartition = new TopicAndPartition(fp.topic, fp.partition);
-				requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(whichTime, 1));
-			}
-
-			int retries = 0;
-			OffsetResponse response;
-			while(true) {
-				kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
-				response = consumer.getOffsetsBefore(request);
-
-				if (response.hasError()) {
-					String exception = "";
-					for (FetchPartition fp : partitions) {
-						short code;
-						if ((code = response.errorCode(fp.topic, fp.partition)) != ErrorMapping.NoError()) {
-							exception += "\nException for topic=" + fp.topic + " partition=" + fp.partition + ": " + StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
-						}
-					}
-					if(++retries >= 3) {
-						throw new RuntimeException("Unable to get last offset for partitions " + partitions + ". " + exception);
-					} else {
-						LOG.warn("Unable to get last offset for partitions: Exception(s): {}", exception);
-					}
-
-				} else {
-					break; // leave retry loop
-				}
-			}
-
-			for (FetchPartition fp: partitions) {
-				// the resulting offset is the next offset we are going to read
-				// for not-yet-consumed partitions, it is 0.
-				fp.nextOffsetToRead = response.offsets(fp.topic, fp.partition)[0];
-			}
-		}
-
-		private static long getInvalidOffsetBehavior(Properties config) {
-			long timeType;
-			String val = config.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "largest");
-			if(val.equals("none")) {
-				throw new RuntimeException("Unable to find previous offset in consumer group. " +
-						"Set 'auto.offset.reset' to 'latest' or 'earliest' to automatically get the offset from Kafka");
-			}
-			if (val.equals("largest") || val.equals("latest")) { // largest is kafka 0.8, latest is kafka 0.9
-				timeType = OffsetRequest.LatestTime();
-			} else {
-				timeType = OffsetRequest.EarliestTime();
-			}
-			return timeType;
-		}
-	}
-
-
-	private static class PartitionInfoFetcher extends Thread {
-
-		private final List<String> topics;
-		private final Properties properties;
-
-		private volatile List<KafkaTopicPartitionLeader> result;
-		private volatile Throwable error;
-
-
-		PartitionInfoFetcher(List<String> topics, Properties properties) {
-			this.topics = topics;
-			this.properties = properties;
-		}
-
-		@Override
-		public void run() {
-			try {
-				result = FlinkKafkaConsumer08.getPartitionsForTopic(topics, properties);
-			}
-			catch (Throwable t) {
-				this.error = t;
-			}
-		}
-
-		public List<KafkaTopicPartitionLeader> getPartitions() throws Exception {
-			try {
-				this.join();
-			}
-			catch (InterruptedException e) {
-				throw new Exception("Partition fetching was cancelled before completion");
-			}
-
-			if (error != null) {
-				throw new Exception("Failed to fetch partitions for topics " + topics.toString(), error);
-			}
-			if (result != null) {
-				return result;
-			}
-			throw new Exception("Partition fetching failed");
-		}
-	}
-
-	private static class KillerWatchDog extends Thread {
-
-		private final Thread toKill;
-		private final long timeout;
-
-		private KillerWatchDog(Thread toKill, long timeout) {
-			super("KillerWatchDog");
-			setDaemon(true);
-
-			this.toKill = toKill;
-			this.timeout = timeout;
-		}
-
-		@SuppressWarnings("deprecation")
-		@Override
-		public void run() {
-			final long deadline = System.currentTimeMillis() + timeout;
-			long now;
-
-			while (toKill.isAlive() && (now = System.currentTimeMillis()) < deadline) {
-				try {
-					toKill.join(deadline - now);
-				}
-				catch (InterruptedException e) {
-					// ignore here, our job is important!
-				}
-			}
-
-			// this is harsh, but this watchdog is a last resort
-			if (toKill.isAlive()) {
-				toKill.stop();
-			}
-		}
-	}
-
-	/**
-	 * Returns a unique list of topics from the topic partition list
-	 *
-	 * @param partitionsList A lost of FetchPartitions's
-	 * @return A unique list of topics from the input map
-	 */
-	public static List<String> getTopics(List<FetchPartition> partitionsList) {
-		HashSet<String> uniqueTopics = new HashSet<>();
-		for (FetchPartition fp: partitionsList) {
-			uniqueTopics.add(fp.topic);
-		}
-		return new ArrayList<>(uniqueTopics);
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java
new file mode 100644
index 0000000..d8d927d
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PartitionInfoFetcher.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
+
+import java.util.List;
+import java.util.Properties;
+
+class PartitionInfoFetcher extends Thread {
+
+	private final List<String> topics;
+	private final Properties properties;
+
+	private volatile List<KafkaTopicPartitionLeader> result;
+	private volatile Throwable error;
+
+
+	PartitionInfoFetcher(List<String> topics, Properties properties) {
+		this.topics = topics;
+		this.properties = properties;
+	}
+
+	@Override
+	public void run() {
+		try {
+			result = FlinkKafkaConsumer08.getPartitionsForTopic(topics, properties);
+		}
+		catch (Throwable t) {
+			this.error = t;
+		}
+	}
+
+	public List<KafkaTopicPartitionLeader> getPartitions() throws Exception {
+		try {
+			this.join();
+		}
+		catch (InterruptedException e) {
+			throw new Exception("Partition fetching was cancelled before completion");
+		}
+
+		if (error != null) {
+			throw new Exception("Failed to fetch partitions for topics " + topics.toString(), error);
+		}
+		if (result != null) {
+			return result;
+		}
+		throw new Exception("Partition fetching failed");
+	}
+}
\ No newline at end of file


[08/14] flink git commit: [FLINK-3375] [kafka connector] Rework/simplify Kafka Connector and have a WatermarkExtractor object per partition

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java
new file mode 100644
index 0000000..99c5d69
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPeriodicWatermarks.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * A special version of the per-kafka-partition-state that additionally holds
+ * a periodic watermark generator (and timestamp extractor) per partition.
+ * 
+ * @param <T> The type of records handled by the watermark generator
+ * @param <KPH> The type of the Kafka partition descriptor, which varies across Kafka versions.
+ */
+public final class KafkaTopicPartitionStateWithPeriodicWatermarks<T, KPH> extends KafkaTopicPartitionState<KPH> {
+	
+	/** The timestamp assigner and watermark generator for the partition */
+	private final AssignerWithPeriodicWatermarks<T> timestampsAndWatermarks;
+	
+	/** The last watermark timestamp generated by this partition */
+	private long partitionWatermark;
+
+	// ------------------------------------------------------------------------
+	
+	public KafkaTopicPartitionStateWithPeriodicWatermarks(
+			KafkaTopicPartition partition, KPH kafkaPartitionHandle,
+			AssignerWithPeriodicWatermarks<T> timestampsAndWatermarks)
+	{
+		super(partition, kafkaPartitionHandle);
+		
+		this.timestampsAndWatermarks = timestampsAndWatermarks;
+		this.partitionWatermark = Long.MIN_VALUE;
+	}
+
+	// ------------------------------------------------------------------------
+	
+	public long getTimestampForRecord (T record) {
+		return timestampsAndWatermarks.extractTimestamp(record, Long.MIN_VALUE);
+	}
+	
+	public long getCurrentWatermarkTimestamp() {
+		Watermark wm = timestampsAndWatermarks.getCurrentWatermark();
+		if (wm != null) {
+			partitionWatermark = Math.max(partitionWatermark, wm.getTimestamp());
+		}
+		return partitionWatermark;
+	}
+
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "KafkaTopicPartitionStateWithPeriodicWatermarks: partition=" + getKafkaTopicPartition()
+				+ ", offset=" + getOffset() + ", watermark=" + partitionWatermark;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java
new file mode 100644
index 0000000..b265990
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateWithPunctuatedWatermarks.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+import javax.annotation.Nullable;
+
+/**
+ * A special version of the per-kafka-partition-state that additionally holds
+ * a periodic watermark generator (and timestamp extractor) per partition.
+ * 
+ * <p>This class is not thread safe, but it gives volatile access to the current
+ * partition watermark ({@link #getCurrentPartitionWatermark()}).
+ * 
+ * @param <T> The type of records handled by the watermark generator
+ * @param <KPH> The type of the Kafka partition descriptor, which varies across Kafka versions
+ */
+public final class KafkaTopicPartitionStateWithPunctuatedWatermarks<T, KPH> extends KafkaTopicPartitionState<KPH> {
+	
+	/** The timestamp assigner and watermark generator for the partition */
+	private final AssignerWithPunctuatedWatermarks<T> timestampsAndWatermarks;
+	
+	/** The last watermark timestamp generated by this partition */
+	private volatile long partitionWatermark;
+
+	// ------------------------------------------------------------------------
+	
+	public KafkaTopicPartitionStateWithPunctuatedWatermarks(
+			KafkaTopicPartition partition, KPH kafkaPartitionHandle,
+			AssignerWithPunctuatedWatermarks<T> timestampsAndWatermarks)
+	{
+		super(partition, kafkaPartitionHandle);
+		
+		this.timestampsAndWatermarks = timestampsAndWatermarks;
+		this.partitionWatermark = Long.MIN_VALUE;
+	}
+
+	// ------------------------------------------------------------------------
+	
+	public long getTimestampForRecord(T record) {
+		return timestampsAndWatermarks.extractTimestamp(record, Long.MIN_VALUE);
+	}
+
+	@Nullable
+	public Watermark checkAndGetNewWatermark(T record, long timestamp) {
+		Watermark mark = timestampsAndWatermarks.checkAndGetNextWatermark(record, timestamp);
+		if (mark != null && mark.getTimestamp() > partitionWatermark) {
+			partitionWatermark = mark.getTimestamp();
+			return mark;
+		}
+		else {
+			return null;
+		}
+	}
+	
+	public long getCurrentPartitionWatermark() {
+		return partitionWatermark;
+	}
+
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "KafkaTopicPartitionStateWithPunctuatedWatermarks: partition=" + getKafkaTopicPartition()
+				+ ", offset=" + getOffset() + ", watermark=" + partitionWatermark;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
index 038f414..37e2ef6 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/partitioner/KafkaPartitioner.java
@@ -14,8 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.streaming.connectors.kafka.partitioner;
 
+package org.apache.flink.streaming.connectors.kafka.partitioner;
 
 import java.io.Serializable;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/util/KafkaUtils.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/util/KafkaUtils.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/util/KafkaUtils.java
index 1be6b00..bda90bd 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/util/KafkaUtils.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/util/KafkaUtils.java
@@ -14,8 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.flink.streaming.connectors.kafka.util;
 
+package org.apache.flink.streaming.connectors.kafka.util;
 
 import java.util.Properties;
 
@@ -38,10 +38,9 @@ public class KafkaUtils {
 					"Entered value='" + config.getProperty(key) + "'. Default value='" + defaultValue + "'");
 		}
 	}
-
-	public static void checkArgument(boolean arg) {
-		if(!arg) {
-			throw new IllegalArgumentException();
-		}
-	}
+	
+	// ------------------------------------------------------------------------
+	
+	/** Private default constructor to prevent instantiation */
+	private KafkaUtils() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
new file mode 100644
index 0000000..f4ef995
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.commons.collections.map.LinkedMap;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+public class FlinkKafkaConsumerBaseTest {
+
+	/**
+	 * Tests that not both types of timestamp extractors / watermark generators can be used.
+	 */
+	@Test
+	public void testEitherWatermarkExtractor() {
+		try {
+			new DummyFlinkKafkaConsumer<>().setPeriodicWatermarkEmitter(null);
+			fail();
+		} catch (NullPointerException ignored) {}
+
+		try {
+			new DummyFlinkKafkaConsumer<>().setPunctuatedWatermarkEmitter(null);
+			fail();
+		} catch (NullPointerException ignored) {}
+		
+		@SuppressWarnings("unchecked")
+		final AssignerWithPeriodicWatermarks<String> periodicAssigner = mock(AssignerWithPeriodicWatermarks.class);
+		@SuppressWarnings("unchecked")
+		final AssignerWithPunctuatedWatermarks<String> punctuatedAssigner = mock(AssignerWithPunctuatedWatermarks.class);
+		
+		DummyFlinkKafkaConsumer<String> c1 = new DummyFlinkKafkaConsumer<>();
+		c1.setPeriodicWatermarkEmitter(periodicAssigner);
+		try {
+			c1.setPunctuatedWatermarkEmitter(punctuatedAssigner);
+			fail();
+		} catch (IllegalStateException ignored) {}
+
+		DummyFlinkKafkaConsumer<String> c2 = new DummyFlinkKafkaConsumer<>();
+		c2.setPunctuatedWatermarkEmitter(punctuatedAssigner);
+		try {
+			c2.setPeriodicWatermarkEmitter(periodicAssigner);
+			fail();
+		} catch (IllegalStateException ignored) {}
+	}
+
+	/**
+	 * Tests that no checkpoints happen when the fetcher is not running.
+	 */
+	@Test
+	public void ignoreCheckpointWhenNotRunning() throws Exception {
+		@SuppressWarnings("unchecked")
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		FlinkKafkaConsumerBase<String> consumer = getConsumer(fetcher, new LinkedMap(), false);
+		assertNull(consumer.snapshotState(17L, 23L));
+		consumer.notifyCheckpointComplete(66L);
+	}
+
+	/**
+	 * Tests that no checkpoints happen when the fetcher is not running.
+	 */
+	@Test
+	public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception {
+		HashMap<KafkaTopicPartition, Long> restoreState = new HashMap<>();
+		restoreState.put(new KafkaTopicPartition("abc", 13), 16768L);
+		restoreState.put(new KafkaTopicPartition("def", 7), 987654321L);
+
+		FlinkKafkaConsumerBase<String> consumer = getConsumer(null, new LinkedMap(), true);
+		consumer.restoreState(restoreState);
+		
+		assertEquals(restoreState, consumer.snapshotState(17L, 23L));
+	}
+
+	/**
+	 * Tests that no checkpoints happen when the fetcher is not running.
+	 */
+	@Test
+	public void checkRestoredNullCheckpointWhenFetcherNotReady() throws Exception {
+		FlinkKafkaConsumerBase<String> consumer = getConsumer(null, new LinkedMap(), true);
+		assertNull(consumer.snapshotState(17L, 23L));
+	}
+	
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testSnapshotState() throws Exception {
+		final HashMap<KafkaTopicPartition, Long> state1 = new HashMap<>();
+		state1.put(new KafkaTopicPartition("abc", 13), 16768L);
+		state1.put(new KafkaTopicPartition("def", 7), 987654321L);
+
+		final HashMap<KafkaTopicPartition, Long> state2 = new HashMap<>();
+		state2.put(new KafkaTopicPartition("abc", 13), 16770L);
+		state2.put(new KafkaTopicPartition("def", 7), 987654329L);
+
+		final HashMap<KafkaTopicPartition, Long> state3 = new HashMap<>();
+		state2.put(new KafkaTopicPartition("abc", 13), 16780L);
+		state2.put(new KafkaTopicPartition("def", 7), 987654377L);
+		
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+		when(fetcher.snapshotCurrentState()).thenReturn(state1, state2, state3);
+			
+		final LinkedMap pendingCheckpoints = new LinkedMap();
+	
+		FlinkKafkaConsumerBase<String> consumer = getConsumer(fetcher, pendingCheckpoints, true);
+		assertEquals(0, pendingCheckpoints.size());
+		
+		// checkpoint 1
+		HashMap<KafkaTopicPartition, Long> snapshot1 = consumer.snapshotState(138L, 19L);
+		assertEquals(state1, snapshot1);
+		assertEquals(1, pendingCheckpoints.size());
+		assertEquals(state1, pendingCheckpoints.get(138L));
+
+		// checkpoint 2
+		HashMap<KafkaTopicPartition, Long> snapshot2 = consumer.snapshotState(140L, 1578L);
+		assertEquals(state2, snapshot2);
+		assertEquals(2, pendingCheckpoints.size());
+		assertEquals(state2, pendingCheckpoints.get(140L));
+		
+		// ack checkpoint 1
+		consumer.notifyCheckpointComplete(138L);
+		assertEquals(1, pendingCheckpoints.size());
+		assertTrue(pendingCheckpoints.containsKey(140L));
+
+		// checkpoint 3
+		HashMap<KafkaTopicPartition, Long> snapshot3 = consumer.snapshotState(141L, 1578L);
+		assertEquals(state3, snapshot3);
+		assertEquals(2, pendingCheckpoints.size());
+		assertEquals(state3, pendingCheckpoints.get(141L));
+		
+		// ack checkpoint 3, subsumes number 2
+		consumer.notifyCheckpointComplete(141L);
+		assertEquals(0, pendingCheckpoints.size());
+
+
+		consumer.notifyCheckpointComplete(666); // invalid checkpoint
+		assertEquals(0, pendingCheckpoints.size());
+
+		// create 500 snapshots
+		for (int i = 100; i < 600; i++) {
+			consumer.snapshotState(i, 15 * i);
+		}
+		assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size());
+
+		// commit only the second last
+		consumer.notifyCheckpointComplete(598);
+		assertEquals(1, pendingCheckpoints.size());
+
+		// access invalid checkpoint
+		consumer.notifyCheckpointComplete(590);
+
+		// and the last
+		consumer.notifyCheckpointComplete(599);
+		assertEquals(0, pendingCheckpoints.size());
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static <T> FlinkKafkaConsumerBase<T> getConsumer(
+			AbstractFetcher<T, ?> fetcher, LinkedMap pendingCheckpoints, boolean running) throws Exception
+	{
+		FlinkKafkaConsumerBase<T> consumer = new DummyFlinkKafkaConsumer<>();
+
+		Field fetcherField = FlinkKafkaConsumerBase.class.getDeclaredField("kafkaFetcher");
+		fetcherField.setAccessible(true);
+		fetcherField.set(consumer, fetcher);
+
+		Field mapField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingCheckpoints");
+		mapField.setAccessible(true);
+		mapField.set(consumer, pendingCheckpoints);
+
+		Field runningField = FlinkKafkaConsumerBase.class.getDeclaredField("running");
+		runningField.setAccessible(true);
+		runningField.set(consumer, running);
+
+		return consumer;
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static final class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
+		private static final long serialVersionUID = 1L;
+
+		@SuppressWarnings("unchecked")
+		public DummyFlinkKafkaConsumer() {
+			super((KeyedDeserializationSchema<T>) mock(KeyedDeserializationSchema.class));
+		}
+
+		@Override
+		protected AbstractFetcher<T, ?> createFetcher(SourceContext<T> sourceContext, List<KafkaTopicPartition> thisSubtaskPartitions, SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic, SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated, StreamingRuntimeContext runtimeContext) throws Exception {
+			return null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
index e86d51a..9beed22 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerPartitionAssignmentTest.java
@@ -18,10 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionLeader;
-import org.apache.kafka.common.Node;
+
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -32,30 +30,27 @@ import java.util.Set;
 
 import static org.junit.Assert.*;
 
-
 /**
  * Tests that the partition assignment is deterministic and stable.
  */
 public class KafkaConsumerPartitionAssignmentTest {
 
-	private final Node fake = new Node(1337, "localhost", 1337);
-
 	@Test
 	public void testPartitionsEqualConsumers() {
 		try {
-			List<KafkaTopicPartitionLeader> inPartitions = new ArrayList<>();
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 4), fake));
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 52), fake));
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 17), fake));
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 1), fake));
+			List<KafkaTopicPartition> inPartitions = Arrays.asList(
+					new KafkaTopicPartition("test-topic", 4),
+					new KafkaTopicPartition("test-topic", 52),
+					new KafkaTopicPartition("test-topic", 17),
+					new KafkaTopicPartition("test-topic", 1));
 
 			for (int i = 0; i < inPartitions.size(); i++) {
-				List<KafkaTopicPartitionLeader> parts = FlinkKafkaConsumerBase.assignPartitions(
-						inPartitions, inPartitions.size(), i);
+				List<KafkaTopicPartition> parts = 
+						FlinkKafkaConsumerBase.assignPartitions(inPartitions, inPartitions.size(), i);
 
 				assertNotNull(parts);
 				assertEquals(1, parts.size());
-				assertTrue(contains(inPartitions, parts.get(0).getTopicPartition().getPartition()));
+				assertTrue(contains(inPartitions, parts.get(0).getPartition()));
 			}
 		}
 		catch (Exception e) {
@@ -64,9 +59,9 @@ public class KafkaConsumerPartitionAssignmentTest {
 		}
 	}
 
-	private boolean contains(List<KafkaTopicPartitionLeader> inPartitions, int partition) {
-		for (KafkaTopicPartitionLeader ktp: inPartitions) {
-			if (ktp.getTopicPartition().getPartition() == partition) {
+	private boolean contains(List<KafkaTopicPartition> inPartitions, int partition) {
+		for (KafkaTopicPartition ktp : inPartitions) {
+			if (ktp.getPartition() == partition) {
 				return true;
 			}
 		}
@@ -78,11 +73,11 @@ public class KafkaConsumerPartitionAssignmentTest {
 		try {
 			final int[] partitionIDs = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14};
 
-			final List<KafkaTopicPartitionLeader> partitions = new ArrayList<>();
-			final Set<KafkaTopicPartitionLeader> allPartitions = new HashSet<>();
+			final List<KafkaTopicPartition> partitions = new ArrayList<>();
+			final Set<KafkaTopicPartition> allPartitions = new HashSet<>();
 
 			for (int p : partitionIDs) {
-				KafkaTopicPartitionLeader part = new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", p), fake);
+				KafkaTopicPartition part = new KafkaTopicPartition("test-topic", p);
 				partitions.add(part);
 				allPartitions.add(part);
 			}
@@ -92,13 +87,14 @@ public class KafkaConsumerPartitionAssignmentTest {
 			final int maxPartitionsPerConsumer = partitions.size() / numConsumers + 1;
 
 			for (int i = 0; i < numConsumers; i++) {
-				List<KafkaTopicPartitionLeader> parts = FlinkKafkaConsumerBase.assignPartitions(partitions, numConsumers, i);
+				List<KafkaTopicPartition> parts = 
+						FlinkKafkaConsumerBase.assignPartitions(partitions, numConsumers, i);
 
 				assertNotNull(parts);
 				assertTrue(parts.size() >= minPartitionsPerConsumer);
 				assertTrue(parts.size() <= maxPartitionsPerConsumer);
 
-				for (KafkaTopicPartitionLeader p : parts) {
+				for (KafkaTopicPartition p : parts) {
 					// check that the element was actually contained
 					assertTrue(allPartitions.remove(p));
 				}
@@ -116,24 +112,24 @@ public class KafkaConsumerPartitionAssignmentTest {
 	@Test
 	public void testPartitionsFewerThanConsumers() {
 		try {
-			List<KafkaTopicPartitionLeader> inPartitions = new ArrayList<>();
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 4), fake));
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 52), fake));
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 17), fake));
-			inPartitions.add(new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", 1), fake));
+			List<KafkaTopicPartition> inPartitions = Arrays.asList(
+					new KafkaTopicPartition("test-topic", 4),
+					new KafkaTopicPartition("test-topic", 52),
+					new KafkaTopicPartition("test-topic", 17),
+					new KafkaTopicPartition("test-topic", 1));
 
-			final Set<KafkaTopicPartitionLeader> allPartitions = new HashSet<>();
+			final Set<KafkaTopicPartition> allPartitions = new HashSet<>();
 			allPartitions.addAll(inPartitions);
 
 			final int numConsumers = 2 * inPartitions.size() + 3;
 
 			for (int i = 0; i < numConsumers; i++) {
-				List<KafkaTopicPartitionLeader> parts = FlinkKafkaConsumerBase.assignPartitions(inPartitions, numConsumers, i);
+				List<KafkaTopicPartition> parts = FlinkKafkaConsumerBase.assignPartitions(inPartitions, numConsumers, i);
 
 				assertNotNull(parts);
 				assertTrue(parts.size() <= 1);
 
-				for (KafkaTopicPartitionLeader p : parts) {
+				for (KafkaTopicPartition p : parts) {
 					// check that the element was actually contained
 					assertTrue(allPartitions.remove(p));
 				}
@@ -151,12 +147,12 @@ public class KafkaConsumerPartitionAssignmentTest {
 	@Test
 	public void testAssignEmptyPartitions() {
 		try {
-			List<KafkaTopicPartitionLeader> ep = new ArrayList<>();
-			List<KafkaTopicPartitionLeader> parts1 = FlinkKafkaConsumerBase.assignPartitions(ep, 4, 2);
+			List<KafkaTopicPartition> ep = new ArrayList<>();
+			List<KafkaTopicPartition> parts1 = FlinkKafkaConsumerBase.assignPartitions(ep, 4, 2);
 			assertNotNull(parts1);
 			assertTrue(parts1.isEmpty());
 
-			List<KafkaTopicPartitionLeader> parts2 = FlinkKafkaConsumerBase.assignPartitions(ep, 1, 0);
+			List<KafkaTopicPartition> parts2 = FlinkKafkaConsumerBase.assignPartitions(ep, 1, 0);
 			assertNotNull(parts2);
 			assertTrue(parts2.isEmpty());
 		}
@@ -170,17 +166,17 @@ public class KafkaConsumerPartitionAssignmentTest {
 	public void testGrowingPartitionsRemainsStable() {
 		try {
 			final int[] newPartitionIDs = {4, 52, 17, 1, 2, 3, 89, 42, 31, 127, 14};
-			List<KafkaTopicPartitionLeader> newPartitions = new ArrayList<>();
+			List<KafkaTopicPartition> newPartitions = new ArrayList<>();
 
 			for (int p : newPartitionIDs) {
-				KafkaTopicPartitionLeader part = new KafkaTopicPartitionLeader(new KafkaTopicPartition("test-topic", p), fake);
+				KafkaTopicPartition part = new KafkaTopicPartition("test-topic", p);
 				newPartitions.add(part);
 			}
 
-			List<KafkaTopicPartitionLeader> initialPartitions = newPartitions.subList(0, 7);
+			List<KafkaTopicPartition> initialPartitions = newPartitions.subList(0, 7);
 
-			final Set<KafkaTopicPartitionLeader> allNewPartitions = new HashSet<>(newPartitions);
-			final Set<KafkaTopicPartitionLeader> allInitialPartitions = new HashSet<>(initialPartitions);
+			final Set<KafkaTopicPartition> allNewPartitions = new HashSet<>(newPartitions);
+			final Set<KafkaTopicPartition> allInitialPartitions = new HashSet<>(initialPartitions);
 
 			final int numConsumers = 3;
 			final int minInitialPartitionsPerConsumer = initialPartitions.size() / numConsumers;
@@ -188,11 +184,11 @@ public class KafkaConsumerPartitionAssignmentTest {
 			final int minNewPartitionsPerConsumer = newPartitions.size() / numConsumers;
 			final int maxNewPartitionsPerConsumer = newPartitions.size() / numConsumers + 1;
 
-			List<KafkaTopicPartitionLeader> parts1 = FlinkKafkaConsumerBase.assignPartitions(
+			List<KafkaTopicPartition> parts1 = FlinkKafkaConsumerBase.assignPartitions(
 					initialPartitions, numConsumers, 0);
-			List<KafkaTopicPartitionLeader> parts2 = FlinkKafkaConsumerBase.assignPartitions(
+			List<KafkaTopicPartition> parts2 = FlinkKafkaConsumerBase.assignPartitions(
 					initialPartitions, numConsumers, 1);
-			List<KafkaTopicPartitionLeader> parts3 = FlinkKafkaConsumerBase.assignPartitions(
+			List<KafkaTopicPartition> parts3 = FlinkKafkaConsumerBase.assignPartitions(
 					initialPartitions, numConsumers, 2);
 
 			assertNotNull(parts1);
@@ -206,15 +202,15 @@ public class KafkaConsumerPartitionAssignmentTest {
 			assertTrue(parts3.size() >= minInitialPartitionsPerConsumer);
 			assertTrue(parts3.size() <= maxInitialPartitionsPerConsumer);
 
-			for (KafkaTopicPartitionLeader p : parts1) {
+			for (KafkaTopicPartition p : parts1) {
 				// check that the element was actually contained
 				assertTrue(allInitialPartitions.remove(p));
 			}
-			for (KafkaTopicPartitionLeader p : parts2) {
+			for (KafkaTopicPartition p : parts2) {
 				// check that the element was actually contained
 				assertTrue(allInitialPartitions.remove(p));
 			}
-			for (KafkaTopicPartitionLeader p : parts3) {
+			for (KafkaTopicPartition p : parts3) {
 				// check that the element was actually contained
 				assertTrue(allInitialPartitions.remove(p));
 			}
@@ -224,11 +220,11 @@ public class KafkaConsumerPartitionAssignmentTest {
 
 			// grow the set of partitions and distribute anew
 
-			List<KafkaTopicPartitionLeader> parts1new = FlinkKafkaConsumerBase.assignPartitions(
+			List<KafkaTopicPartition> parts1new = FlinkKafkaConsumerBase.assignPartitions(
 					newPartitions, numConsumers, 0);
-			List<KafkaTopicPartitionLeader> parts2new = FlinkKafkaConsumerBase.assignPartitions(
+			List<KafkaTopicPartition> parts2new = FlinkKafkaConsumerBase.assignPartitions(
 					newPartitions, numConsumers, 1);
-			List<KafkaTopicPartitionLeader> parts3new = FlinkKafkaConsumerBase.assignPartitions(
+			List<KafkaTopicPartition> parts3new = FlinkKafkaConsumerBase.assignPartitions(
 					newPartitions, numConsumers, 2);
 
 			// new partitions must include all old partitions
@@ -248,15 +244,15 @@ public class KafkaConsumerPartitionAssignmentTest {
 			assertTrue(parts3new.size() >= minNewPartitionsPerConsumer);
 			assertTrue(parts3new.size() <= maxNewPartitionsPerConsumer);
 
-			for (KafkaTopicPartitionLeader p : parts1new) {
+			for (KafkaTopicPartition p : parts1new) {
 				// check that the element was actually contained
 				assertTrue(allNewPartitions.remove(p));
 			}
-			for (KafkaTopicPartitionLeader p : parts2new) {
+			for (KafkaTopicPartition p : parts2new) {
 				// check that the element was actually contained
 				assertTrue(allNewPartitions.remove(p));
 			}
-			for (KafkaTopicPartitionLeader p : parts3new) {
+			for (KafkaTopicPartition p : parts3new) {
 				// check that the element was actually contained
 				assertTrue(allNewPartitions.remove(p));
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index 340950b..aa5344b 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -26,8 +26,6 @@ import kafka.javaapi.consumer.ConsumerConnector;
 import kafka.message.MessageAndMetadata;
 import kafka.server.KafkaServer;
 
-import org.apache.commons.collections.map.LinkedMap;
-
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobExecutionResult;
@@ -66,12 +64,10 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
 import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators;
 import org.apache.flink.streaming.connectors.kafka.testutils.DiscardingSink;
 import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils;
-import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
 import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2Partitioner;
@@ -98,7 +94,6 @@ import org.junit.Rule;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
@@ -176,70 +171,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			}
 		}
 	}
-	/**
-	 * Test that validates that checkpointing and checkpoint notification works properly
-	 */
-	public void runCheckpointingTest() throws Exception {
-		createTestTopic("testCheckpointing", 1, 1);
-
-		FlinkKafkaConsumerBase<String> source = kafkaServer.getConsumer("testCheckpointing", new SimpleStringSchema(), standardProps);
-		Field pendingCheckpointsField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingCheckpoints");
-		pendingCheckpointsField.setAccessible(true);
-		LinkedMap pendingCheckpoints = (LinkedMap) pendingCheckpointsField.get(source);
-
-		Assert.assertEquals(0, pendingCheckpoints.size());
-		source.setRuntimeContext(new MockRuntimeContext(1, 0));
-
-		final HashMap<KafkaTopicPartition, Long> initialOffsets = new HashMap<>();
-		initialOffsets.put(new KafkaTopicPartition("testCheckpointing", 0), 1337L);
-
-		// first restore
-		source.restoreState(initialOffsets);
-
-		// then open
-		source.open(new Configuration());
-		HashMap<KafkaTopicPartition, Long> state1 = source.snapshotState(1, 15);
-
-		assertEquals(initialOffsets, state1);
-
-		HashMap<KafkaTopicPartition, Long> state2 = source.snapshotState(2, 30);
-		Assert.assertEquals(initialOffsets, state2);
-
-		Assert.assertEquals(2, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(1);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(2);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		source.notifyCheckpointComplete(666); // invalid checkpoint
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		// create 500 snapshots
-		for (int i = 100; i < 600; i++) {
-			source.snapshotState(i, 15 * i);
-		}
-		Assert.assertEquals(FlinkKafkaConsumerBase.MAX_NUM_PENDING_CHECKPOINTS, pendingCheckpoints.size());
-
-		// commit only the second last
-		source.notifyCheckpointComplete(598);
-		Assert.assertEquals(1, pendingCheckpoints.size());
-
-		// access invalid checkpoint
-		source.notifyCheckpointComplete(590);
-
-		// and the last
-		source.notifyCheckpointComplete(599);
-		Assert.assertEquals(0, pendingCheckpoints.size());
-
-		source.close();
-
-		deleteTestTopic("testCheckpointing");
-	}
-
-
-
+	
 	/**
 	 * Ensure Kafka is working on both producer and consumer side.
 	 * This executes a job that contains two Flink pipelines.
@@ -409,7 +341,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
 		env.enableCheckpointing(500);
 		env.setParallelism(parallelism);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000));
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0));
 		env.getConfig().disableSysoutLogging();
 
 		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps);
@@ -454,7 +386,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
 		env.enableCheckpointing(500);
 		env.setParallelism(parallelism);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000));
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0));
 		env.getConfig().disableSysoutLogging();
 
 		FlinkKafkaConsumerBase<Integer> kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps);
@@ -499,7 +431,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
 		env.enableCheckpointing(500);
 		env.setParallelism(parallelism);
-		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 1000));
+		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 0));
 		env.getConfig().disableSysoutLogging();
 		env.setBufferTimeout(0);
 
@@ -562,7 +494,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		runnerThread.start();
 
 		// wait a bit before canceling
-		Thread.sleep(8000);
+		Thread.sleep(2000);
 
 		Throwable failueCause = jobError.get();
 		if(failueCause != null) {
@@ -634,10 +566,10 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		runnerThread.start();
 
 		// wait a bit before canceling
-		Thread.sleep(8000);
+		Thread.sleep(2000);
 
 		Throwable failueCause = error.get();
-		if(failueCause != null) {
+		if (failueCause != null) {
 			failueCause.printStackTrace();
 			Assert.fail("Test failed prematurely with: " + failueCause.getMessage());
 		}
@@ -709,7 +641,8 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		final int NUM_ELEMENTS = 20;
 
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-
+		env.getConfig().disableSysoutLogging();
+		
 		// create topics with content
 		final List<String> topics = new ArrayList<>();
 		for (int i = 0; i < NUM_TOPICS; i++) {
@@ -745,6 +678,8 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 		// run second job consuming from multiple topics
 		env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.getConfig().disableSysoutLogging();
+		
 		stream = env.addSource(kafkaServer.getConsumer(topics, schema, standardProps));
 
 		stream.flatMap(new FlatMapFunction<Tuple3<Integer, Integer, String>, Integer>() {
@@ -1453,50 +1388,50 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 
 	/////////////			Testing the Kafka consumer with embeded watermark generation functionality			///////////////
 
-	@RetryOnException(times=0, exception=kafka.common.NotLeaderForPartitionException.class)
-	public void runExplicitPunctuatedWMgeneratingConsumerTest(boolean emptyPartition) throws Exception {
-
-		final String topic1 = "wmExtractorTopic1_" + UUID.randomUUID().toString();
-		final String topic2 = "wmExtractorTopic2_" + UUID.randomUUID().toString();
-
-		final Map<String, Boolean> topics = new HashMap<>();
-		topics.put(topic1, false);
-		topics.put(topic2, emptyPartition);
-
-		final int noOfTopcis = topics.size();
-		final int partitionsPerTopic = 1;
-		final int elementsPerPartition = 100 + 1;
-
-		final int totalElements = emptyPartition ?
-			partitionsPerTopic * elementsPerPartition :
-			noOfTopcis * partitionsPerTopic * elementsPerPartition;
-
-		createTestTopic(topic1, partitionsPerTopic, 1);
-		createTestTopic(topic2, partitionsPerTopic, 1);
-
-		final StreamExecutionEnvironment env =
-			StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-		env.setParallelism(partitionsPerTopic);
-		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
-		env.getConfig().disableSysoutLogging();
-
-		TypeInformation<Tuple2<Long, Integer>> longIntType = TypeInfoParser.parse("Tuple2<Long, Integer>");
-
-		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
-		producerProperties.setProperty("retries", "0");
-
-		putDataInTopics(env, producerProperties, elementsPerPartition, topics, longIntType);
-
-		List<String> topicTitles = new ArrayList<>(topics.keySet());
-		runPunctuatedComsumer(env, topicTitles, totalElements, longIntType);
-
-		executeAndCatchException(env, "runComsumerWithPunctuatedExplicitWMTest");
-
-		for(String topic: topicTitles) {
-			deleteTestTopic(topic);
-		}
-	}
+//	@RetryOnException(times=0, exception=kafka.common.NotLeaderForPartitionException.class)
+//	public void runExplicitPunctuatedWMgeneratingConsumerTest(boolean emptyPartition) throws Exception {
+//
+//		final String topic1 = "wmExtractorTopic1_" + UUID.randomUUID().toString();
+//		final String topic2 = "wmExtractorTopic2_" + UUID.randomUUID().toString();
+//
+//		final Map<String, Boolean> topics = new HashMap<>();
+//		topics.put(topic1, false);
+//		topics.put(topic2, emptyPartition);
+//
+//		final int noOfTopcis = topics.size();
+//		final int partitionsPerTopic = 1;
+//		final int elementsPerPartition = 100 + 1;
+//
+//		final int totalElements = emptyPartition ?
+//			partitionsPerTopic * elementsPerPartition :
+//			noOfTopcis * partitionsPerTopic * elementsPerPartition;
+//
+//		createTestTopic(topic1, partitionsPerTopic, 1);
+//		createTestTopic(topic2, partitionsPerTopic, 1);
+//
+//		final StreamExecutionEnvironment env =
+//			StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+//		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+//		env.setParallelism(partitionsPerTopic);
+//		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
+//		env.getConfig().disableSysoutLogging();
+//
+//		TypeInformation<Tuple2<Long, Integer>> longIntType = TypeInfoParser.parse("Tuple2<Long, Integer>");
+//
+//		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+//		producerProperties.setProperty("retries", "0");
+//
+//		putDataInTopics(env, producerProperties, elementsPerPartition, topics, longIntType);
+//
+//		List<String> topicTitles = new ArrayList<>(topics.keySet());
+//		runPunctuatedComsumer(env, topicTitles, totalElements, longIntType);
+//
+//		executeAndCatchException(env, "runComsumerWithPunctuatedExplicitWMTest");
+//
+//		for(String topic: topicTitles) {
+//			deleteTestTopic(topic);
+//		}
+//	}
 
 	private void executeAndCatchException(StreamExecutionEnvironment env, String execName) throws Exception {
 		try {

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
index e251174..14e74f1 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
@@ -167,7 +167,7 @@ public abstract class KafkaProducerTestBase extends KafkaTestBase {
 
 	// ------------------------------------------------------------------------
 
-	public static class CustomPartitioner extends KafkaPartitioner implements Serializable {
+	public static class CustomPartitioner extends KafkaPartitioner<Tuple2<Long, String>> implements Serializable {
 
 		private final int expectedPartitions;
 
@@ -177,12 +177,10 @@ public abstract class KafkaProducerTestBase extends KafkaTestBase {
 
 
 		@Override
-		public int partition(Object next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
-			Tuple2<Long, String> tuple = (Tuple2<Long, String>) next;
-
+		public int partition(Tuple2<Long, String> next, byte[] serializedKey, byte[] serializedValue, int numPartitions) {
 			assertEquals(expectedPartitions, numPartitions);
 
-			return (int) (tuple.f0 % numPartitions);
+			return (int) (next.f0 % numPartitions);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
index 9f8159c..9e3c33b 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
@@ -26,16 +26,16 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.api.transformations.SourceTransformation;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.InstantiationUtil;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,14 +44,18 @@ import java.io.Serializable;
 import java.util.Properties;
 
 import static org.apache.flink.test.util.TestUtils.tryExecute;
-
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 /**
  * A class containing a special Kafka broker which has a log retention of only 250 ms.
  * This way, we can make sure our consumer is properly handling cases where we run into out of offset
  * errors
  */
+@SuppressWarnings("serial")
 public class KafkaShortRetentionTestBase implements Serializable {
+	
 	protected static final Logger LOG = LoggerFactory.getLogger(KafkaShortRetentionTestBase.class);
+	
 	private static KafkaTestEnvironment kafkaServer;
 	private static Properties standardProps;
 	private static ForkableFlinkMiniCluster flink;
@@ -108,7 +112,7 @@ public class KafkaShortRetentionTestBase implements Serializable {
 		final String topic = "auto-offset-reset-test";
 
 		final int parallelism = 1;
-		final int elementsPerPartition = 50000; // with a sleep time of 1 ms per element, test should run for 50 s
+		final int elementsPerPartition = 50000;
 
 		Properties tprops = new Properties();
 		tprops.setProperty("retention.ms", "250");
@@ -162,6 +166,7 @@ public class KafkaShortRetentionTestBase implements Serializable {
 		kafkaServer.deleteTestTopic(topic);
 	}
 
+	
 	private class NonContinousOffsetsDeserializationSchema implements KeyedDeserializationSchema<String> {
 		private int numJumps;
 		long nextExpected = 0;
@@ -205,12 +210,8 @@ public class KafkaShortRetentionTestBase implements Serializable {
 	 */
 	public void runFailOnAutoOffsetResetNone() throws Exception {
 		final String topic = "auto-offset-reset-none-test";
-
 		final int parallelism = 1;
-		final int elementsPerPartition = 50000; // with a sleep time of 1 ms per element, test should run for 50 s
-		final int totalElements = parallelism * elementsPerPartition;
-
-
+		
 		kafkaServer.createTestTopic(topic, parallelism, 1);
 
 		final StreamExecutionEnvironment env =
@@ -218,8 +219,7 @@ public class KafkaShortRetentionTestBase implements Serializable {
 		env.setParallelism(parallelism);
 		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
 		env.getConfig().disableSysoutLogging();
-
-
+		
 		// ----------- add consumer ----------
 
 		Properties customProps = new Properties();
@@ -245,4 +245,27 @@ public class KafkaShortRetentionTestBase implements Serializable {
 		kafkaServer.deleteTestTopic(topic);
 	}
 
+	public void runFailOnAutoOffsetResetNoneEager() throws Exception {
+		final String topic = "auto-offset-reset-none-test";
+		final int parallelism = 1;
+
+		kafkaServer.createTestTopic(topic, parallelism, 1);
+
+		// ----------- add consumer ----------
+
+		Properties customProps = new Properties();
+		customProps.putAll(standardProps);
+		customProps.setProperty("auto.offset.reset", "none"); // test that "none" leads to an exception
+		
+		try {
+			kafkaServer.getConsumer(topic, new SimpleStringSchema(), customProps);
+			fail("should fail with an exception");
+		}
+		catch (IllegalArgumentException e) {
+			// expected
+			assertTrue(e.getMessage().contains("none"));
+		}
+
+		kafkaServer.deleteTestTopic(topic);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java
new file mode 100644
index 0000000..0e16263
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+
+import static org.junit.Assert.*;
+
+public class KafkaTopicPartitionTest {
+	
+	@Test
+	public void validateUid() {
+		Field uidField;
+		try {
+			uidField = KafkaTopicPartition.class.getDeclaredField("serialVersionUID");
+			uidField.setAccessible(true);
+		}
+		catch (NoSuchFieldException e) {
+			fail("serialVersionUID is not defined");
+			return;
+		}
+		
+		assertTrue(Modifier.isStatic(uidField.getModifiers()));
+		assertTrue(Modifier.isFinal(uidField.getModifiers()));
+		assertTrue(Modifier.isPrivate(uidField.getModifiers()));
+		
+		assertEquals(long.class, uidField.getType());
+		
+		// the UID has to be constant to make sure old checkpoints/savepoints can be read 
+		try {
+			assertEquals(722083576322742325L, uidField.getLong(null));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
index e94adb5..24822ed 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
@@ -36,30 +36,39 @@ public class JobManagerCommunicationUtils {
 	
 	
 	public static void cancelCurrentJob(ActorGateway jobManager) throws Exception {
+		JobStatusMessage status = null;
 		
-		// find the jobID
-		Future<Object> listResponse = jobManager.ask(
-				JobManagerMessages.getRequestRunningJobsStatus(),
-				askTimeout);
-
-		List<JobStatusMessage> jobs;
-		try {
-			Object result = Await.result(listResponse, askTimeout);
-			jobs = ((JobManagerMessages.RunningJobsStatus) result).getStatusMessages();
-		}
-		catch (Exception e) {
-			throw new Exception("Could not cancel job - failed to retrieve running jobs from the JobManager.", e);
-		}
+		for (int i = 0; i < 200; i++) {
+			// find the jobID
+			Future<Object> listResponse = jobManager.ask(
+					JobManagerMessages.getRequestRunningJobsStatus(),
+					askTimeout);
+	
+			List<JobStatusMessage> jobs;
+			try {
+				Object result = Await.result(listResponse, askTimeout);
+				jobs = ((JobManagerMessages.RunningJobsStatus) result).getStatusMessages();
+			}
+			catch (Exception e) {
+				throw new Exception("Could not cancel job - failed to retrieve running jobs from the JobManager.", e);
+			}
 		
-		if (jobs.isEmpty()) {
-			throw new Exception("Could not cancel job - no running jobs");
-		}
-		if (jobs.size() != 1) {
-			throw new Exception("Could not cancel job - more than one running job.");
+			if (jobs.isEmpty()) {
+				// try again, fall through the loop
+				Thread.sleep(50);
+			}
+			else if (jobs.size() == 1) {
+				status = jobs.get(0);
+			}
+			else {
+				throw new Exception("Could not cancel job - more than one running job.");
+			}
 		}
 		
-		JobStatusMessage status = jobs.get(0);
-		if (status.getJobState().isTerminalState()) {
+		if (status == null) {
+			throw new Exception("Could not cancel job - no running jobs");	
+		}
+		else if (status.getJobState().isTerminalState()) {
 			throw new Exception("Could not cancel job - job is not running any more");
 		}
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/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 17e2e6f..e74eee4 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
@@ -28,7 +28,6 @@ import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.state.OperatorState;
 import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.state.ValueState;
@@ -44,6 +43,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+@SuppressWarnings("deprecation")
 public class MockRuntimeContext extends StreamingRuntimeContext {
 
 	private final int numberOfParallelSubtasks;
@@ -57,15 +57,6 @@ public class MockRuntimeContext extends StreamingRuntimeContext {
 		this.indexOfThisSubtask = indexOfThisSubtask;
 	}
 
-	private static class MockStreamOperator extends AbstractStreamOperator<Integer> {
-		private static final long serialVersionUID = -1153976702711944427L;
-
-		@Override
-		public ExecutionConfig getExecutionConfig() {
-			return new ExecutionConfig();
-		}
-	}
-
 	@Override
 	public boolean isCheckpointingEnabled() {
 		return true;
@@ -152,12 +143,12 @@ public class MockRuntimeContext extends StreamingRuntimeContext {
 	}
 
 	@Override
-	public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
+	public <S> org.apache.flink.api.common.state.OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
 		throw new UnsupportedOperationException();
 	}
 
 	@Override
-	public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
+	public <S> org.apache.flink.api.common.state.OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
 		throw new UnsupportedOperationException();
 	}
 
@@ -175,4 +166,15 @@ public class MockRuntimeContext extends StreamingRuntimeContext {
 	public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
 		throw new UnsupportedOperationException();
 	}
+	
+	// ------------------------------------------------------------------------
+
+	private static class MockStreamOperator extends AbstractStreamOperator<Integer> {
+		private static final long serialVersionUID = -1153976702711944427L;
+
+		@Override
+		public ExecutionConfig getExecutionConfig() {
+			return new ExecutionConfig();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
index 4b17300..4388c9d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.api.functions;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.streaming.api.watermark.Watermark;
 
+import javax.annotation.Nullable;
+
 /**
  * The {@code AssignerWithPeriodicWatermarks} assigns event time timestamps to elements,
  * and generates low watermarks that signal event time progress within the stream.
@@ -71,5 +73,6 @@ public interface AssignerWithPeriodicWatermarks<T> extends TimestampAssigner<T>
 	 *
 	 * @return {@code Null}, if no watermark should be emitted, or the next watermark to emit.
 	 */
+	@Nullable
 	Watermark getCurrentWatermark();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPunctuatedWatermarks.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPunctuatedWatermarks.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPunctuatedWatermarks.java
index 48f29b2..5b5694c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPunctuatedWatermarks.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPunctuatedWatermarks.java
@@ -20,6 +20,8 @@ package org.apache.flink.streaming.api.functions;
 
 import org.apache.flink.streaming.api.watermark.Watermark;
 
+import javax.annotation.Nullable;
+
 /**
  * The {@code AssignerWithPunctuatedWatermarks} assigns event time timestamps to elements,
  * and generates low watermarks that signal event time progress within the stream.
@@ -79,5 +81,6 @@ public interface AssignerWithPunctuatedWatermarks<T> extends TimestampAssigner<T
 	 *
 	 * @return {@code Null}, if no watermark should be emitted, or the next watermark to emit.
 	 */
+	@Nullable
 	Watermark checkAndGetNextWatermark(T lastElement, long extractedTimestamp);
 }


[05/14] flink git commit: [FLINK-3375] [kafka connector] Add per-Kafka-partition watermark generation to the FlinkKafkaConsumer

Posted by se...@apache.org.
[FLINK-3375] [kafka connector] Add per-Kafka-partition watermark generation to the FlinkKafkaConsumer

This closes #1839


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

Branch: refs/heads/master
Commit: 0ac1549ec58c1737a79e5770a171a8b14bed56dc
Parents: 885d543
Author: kl0u <kk...@gmail.com>
Authored: Tue Mar 8 17:35:14 2016 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 .../connectors/kafka/FlinkKafkaConsumer08.java  |  56 +--
 .../connectors/kafka/internals/Fetcher.java     |  12 +-
 .../kafka/internals/LegacyFetcher.java          |  51 ++-
 .../kafka/internals/ZookeeperOffsetHandler.java |   4 +-
 .../connectors/kafka/Kafka08ITCase.java         |  16 +-
 .../connectors/kafka/KafkaConsumerTest.java     |  12 +-
 .../connectors/kafka/FlinkKafkaConsumer09.java  |  13 +-
 .../connectors/kafka/Kafka09ITCase.java         |  11 +-
 .../kafka/FlinkKafkaConsumerBase.java           | 357 ++++++++++++++++++-
 .../kafka/internals/KafkaPartitionState.java    |  65 ++++
 .../connectors/kafka/KafkaConsumerTestBase.java | 234 +++++++++++-
 .../AssignerWithPeriodicWatermarks.java         |   2 +-
 12 files changed, 744 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
index 865fe36..4748781 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
@@ -28,6 +28,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.kafka.internals.Fetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaPartitionState;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
 import org.apache.flink.streaming.connectors.kafka.internals.LegacyFetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.OffsetHandler;
@@ -108,11 +109,6 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer08.class);
 
-	/** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid),
-	 * and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */
-	public static final long OFFSET_NOT_SET = -915623761776L;
-
-
 	/** Configuration key for the number of retries for getting the partition info */
 	public static final String GET_PARTITIONS_RETRIES_KEY = "flink.get-partitions.retry";
 
@@ -252,17 +248,19 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 			this.fetcher = null; // fetcher remains null
 			return;
 		}
-		
 
 		// offset handling
 		offsetHandler = new ZookeeperOffsetHandler(props);
 
 		committedOffsets = new HashMap<>();
 
-		Map<KafkaTopicPartition, Long> subscribedPartitionsWithOffsets = new HashMap<>(subscribedPartitions.size());
-		// initially load the map with "offset not set"
+		// initially load the map with "offset not set", last max read timestamp set to Long.MIN_VALUE
+		// and mark the partition as in-active, until we receive the first element
+		Map<KafkaTopicPartition, KafkaPartitionState> subscribedPartitionsWithOffsets =
+			new HashMap<>(subscribedPartitions.size());
 		for(KafkaTopicPartition ktp: subscribedPartitions) {
-			subscribedPartitionsWithOffsets.put(ktp, FlinkKafkaConsumer08.OFFSET_NOT_SET);
+			subscribedPartitionsWithOffsets.put(ktp,
+				new KafkaPartitionState(ktp.getPartition(), FlinkKafkaConsumerBase.OFFSET_NOT_SET));
 		}
 
 		// seek to last known pos, from restore request
@@ -272,16 +270,20 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 						thisConsumerIndex, KafkaTopicPartition.toString(restoreToOffset));
 			}
 			// initialize offsets with restored state
-			this.offsetsState = restoreToOffset;
-			subscribedPartitionsWithOffsets.putAll(restoreToOffset);
+			this.partitionState = restoreInfoFromCheckpoint();
+			subscribedPartitionsWithOffsets.putAll(partitionState);
 			restoreToOffset = null;
 		}
 		else {
-			// start with empty offsets
-			offsetsState = new HashMap<>();
+			// start with empty partition state
+			partitionState = new HashMap<>();
 
 			// no restore request: overwrite offsets.
-			subscribedPartitionsWithOffsets.putAll(offsetHandler.getOffsets(subscribedPartitions));
+			for(Map.Entry<KafkaTopicPartition, Long> offsetInfo: offsetHandler.getOffsets(subscribedPartitions).entrySet()) {
+				KafkaTopicPartition key = offsetInfo.getKey();
+				subscribedPartitionsWithOffsets.put(key,
+					new KafkaPartitionState(key.getPartition(), offsetInfo.getValue()));
+			}
 		}
 		if(subscribedPartitionsWithOffsets.size() != subscribedPartitions.size()) {
 			throw new IllegalStateException("The subscribed partitions map has more entries than the subscribed partitions " +
@@ -289,22 +291,22 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 		}
 
 		// create fetcher
-		fetcher = new LegacyFetcher(subscribedPartitionsWithOffsets, props,
+		fetcher = new LegacyFetcher<T>(this, subscribedPartitionsWithOffsets, props,
 				getRuntimeContext().getTaskName(), getRuntimeContext().getUserCodeClassLoader());
 	}
 
 	@Override
 	public void run(SourceContext<T> sourceContext) throws Exception {
 		if (fetcher != null) {
-			// For non-checkpointed sources, a thread which periodically commits the current offset into ZK.
-			PeriodicOffsetCommitter<T> offsetCommitter = null;
-
-			// check whether we need to start the periodic checkpoint committer
 			StreamingRuntimeContext streamingRuntimeContext = (StreamingRuntimeContext) getRuntimeContext();
+
+			// if we have a non-checkpointed source, start a thread which periodically commits
+			// the current offset into ZK.
+
+			PeriodicOffsetCommitter<T> offsetCommitter = null;
 			if (!streamingRuntimeContext.isCheckpointingEnabled()) {
 				// we use Kafka's own configuration parameter key for this.
-				// Note that the default configuration value in Kafka is 60 * 1000, so we use the
-				// same here.
+				// Note that the default configuration value in Kafka is 60 * 1000, so we use the same here.
 				long commitInterval = getLongFromConfig(props, "auto.commit.interval.ms", 60000);
 				offsetCommitter = new PeriodicOffsetCommitter<>(commitInterval, this);
 				offsetCommitter.setDaemon(true);
@@ -313,7 +315,7 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 			}
 
 			try {
-				fetcher.run(sourceContext, deserializer, offsetsState);
+				fetcher.run(sourceContext, deserializer, partitionState);
 			} finally {
 				if (offsetCommitter != null) {
 					offsetCommitter.close();
@@ -439,7 +441,7 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 		private final FlinkKafkaConsumer08<T> consumer;
 		private volatile boolean running = true;
 
-		public PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer08<T> consumer) {
+		PeriodicOffsetCommitter(long commitInterval, FlinkKafkaConsumer08<T> consumer) {
 			this.commitInterval = commitInterval;
 			this.consumer = consumer;
 		}
@@ -453,9 +455,12 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 						Thread.sleep(commitInterval);
 						//  ------------  commit current offsets ----------------
 
-						// create copy of current offsets
+						// create copy a deep copy of the current offsets
 						@SuppressWarnings("unchecked")
-						HashMap<KafkaTopicPartition, Long> currentOffsets = (HashMap<KafkaTopicPartition, Long>) consumer.offsetsState.clone();
+						HashMap<KafkaTopicPartition, Long> currentOffsets = new HashMap<>(consumer.partitionState.size());
+						for (Map.Entry<KafkaTopicPartition, KafkaPartitionState> entry: consumer.partitionState.entrySet()) {
+							currentOffsets.put(entry.getKey(), entry.getValue().getOffset());
+						}
 						consumer.commitOffsets(currentOffsets);
 					} catch (InterruptedException e) {
 						if (running) {
@@ -474,7 +479,6 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
 			this.running = false;
 			this.interrupt();
 		}
-
 	}
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
index 2dacbce..f86687e 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Fetcher.java
@@ -39,8 +39,10 @@ public interface Fetcher {
 	/**
 	 * Starts fetch data from Kafka and emitting it into the stream.
 	 * 
-	 * <p>To provide exactly once guarantees, the fetcher needs emit a record and update the update
-	 * of the last consumed offset in one atomic operation:</p>
+	 * <p>To provide exactly once guarantees, the fetcher needs emit a record and update the last
+	 * consumed offset in one atomic operation. This is done in the
+	 * {@link org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase#processElement(SourceFunction.SourceContext, KafkaTopicPartition, Object, long)}
+	 * which is called from within the {@link SourceFunction.SourceContext#getCheckpointLock()}, as shown below:</p>
 	 * <pre>{@code
 	 * 
 	 * while (running) {
@@ -48,8 +50,7 @@ public interface Fetcher {
 	 *     long offset = ...
 	 *     int partition = ...
 	 *     synchronized (sourceContext.getCheckpointLock()) {
-	 *         sourceContext.collect(next);
-	 *         lastOffsets[partition] = offset;
+	 *         processElement(sourceContext, partition, next, offset)
 	 *     }
 	 * }
 	 * }</pre>
@@ -60,8 +61,7 @@ public interface Fetcher {
 	 * @param lastOffsets The map into which to store the offsets for which elements are emitted (operator state)
 	 */
 	<T> void run(SourceFunction.SourceContext<T> sourceContext, KeyedDeserializationSchema<T> valueDeserializer,
-				HashMap<KafkaTopicPartition, Long> lastOffsets) throws Exception;
-
+				HashMap<KafkaTopicPartition, KafkaPartitionState> lastOffsets) throws Exception;
 
 	/**
 	 * Exit run loop with given error and release all resources.

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
index fa3b6a8..c4dd55c 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
@@ -30,6 +30,7 @@ import kafka.message.MessageAndOffset;
 
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.StringUtils;
@@ -61,13 +62,12 @@ import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.getInt
  * 
  * <p>This code is in parts based on the tutorial code for the low-level Kafka consumer.</p>
  */
-public class LegacyFetcher implements Fetcher {
+public class LegacyFetcher<T> implements Fetcher {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(LegacyFetcher.class);
 
 	private static final FetchPartition MARKER = new FetchPartition("n/a", -1, -1);
-	
-	
+
 	/** The properties that configure the Kafka connection */
 	private final Properties config;
 	
@@ -79,7 +79,6 @@ public class LegacyFetcher implements Fetcher {
 	
 	/** The classloader for dynamically loaded classes */
 	private final ClassLoader userCodeClassloader;
-
 	
 	/** Reference the the thread that executed the run() method. */
 	private volatile Thread mainThread;
@@ -94,6 +93,9 @@ public class LegacyFetcher implements Fetcher {
 	 */
 	private final ClosableBlockingQueue<FetchPartition> unassignedPartitions = new ClosableBlockingQueue<>();
 
+	/** The {@link FlinkKafkaConsumer08} to whom this Fetcher belongs. */
+	private final FlinkKafkaConsumer08<T> flinkKafkaConsumer;
+
 	/**
 	 * Create a LegacyFetcher instance.
 	 *
@@ -103,20 +105,23 @@ public class LegacyFetcher implements Fetcher {
 	 * @param userCodeClassloader classloader for loading user code
 	 */
 	public LegacyFetcher(
-				Map<KafkaTopicPartition, Long> initialPartitionsToRead, Properties props,
+				FlinkKafkaConsumer08<T> owner,
+				Map<KafkaTopicPartition, KafkaPartitionState> initialPartitionsToRead,
+				Properties props,
 				String taskName, ClassLoader userCodeClassloader) {
-		
+
+		this.flinkKafkaConsumer = requireNonNull(owner);
 		this.config = requireNonNull(props, "The config properties cannot be null");
 		this.userCodeClassloader = requireNonNull(userCodeClassloader);
 		if (initialPartitionsToRead.size() == 0) {
 			throw new IllegalArgumentException("List of initial partitions is empty");
 		}
 
-		for (Map.Entry<KafkaTopicPartition, Long> partitionToRead: initialPartitionsToRead.entrySet()) {
+		for (Map.Entry<KafkaTopicPartition, KafkaPartitionState> partitionToRead: initialPartitionsToRead.entrySet()) {
 			KafkaTopicPartition ktp = partitionToRead.getKey();
 			// we increment the offset by one so that we fetch the next message in the partition.
-			long offset = partitionToRead.getValue();
-			if (offset >= 0 && offset != FlinkKafkaConsumer08.OFFSET_NOT_SET) {
+			long offset = partitionToRead.getValue().getOffset();
+			if (offset >= 0 && offset != FlinkKafkaConsumerBase.OFFSET_NOT_SET) {
 				offset += 1L;
 			}
 			unassignedPartitions.add(new FetchPartition(ktp.getTopic(), ktp.getPartition(), offset));
@@ -141,7 +146,7 @@ public class LegacyFetcher implements Fetcher {
 	@Override
 	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
 						KeyedDeserializationSchema<T> deserializer,
-						HashMap<KafkaTopicPartition, Long> lastOffsets) throws Exception {
+						HashMap<KafkaTopicPartition, KafkaPartitionState> partitionState) throws Exception {
 
 		// NOTE: This method needs to always release all resources it acquires
 
@@ -173,7 +178,7 @@ public class LegacyFetcher implements Fetcher {
 							
 							if (brokerThread == null || !brokerThread.getNewPartitionsQueue().isOpen()) {
 								// start new thread
-								brokerThread = createAndStartSimpleConsumerThread(sourceContext, deserializer, lastOffsets, partitions, leader);
+								brokerThread = createAndStartSimpleConsumerThread(sourceContext, deserializer, partitions, leader);
 								brokerToThread.put(leader, brokerThread);
 							} else {
 								// put elements into queue of thread
@@ -185,7 +190,7 @@ public class LegacyFetcher implements Fetcher {
 										// create a new thread for connecting to this broker
 										List<FetchPartition> seedPartitions = new ArrayList<>();
 										seedPartitions.add(fp);
-										brokerThread = createAndStartSimpleConsumerThread(sourceContext, deserializer, lastOffsets, seedPartitions, leader);
+										brokerThread = createAndStartSimpleConsumerThread(sourceContext, deserializer, seedPartitions, leader);
 										brokerToThread.put(leader, brokerThread);
 										newPartitionsQueue = brokerThread.getNewPartitionsQueue(); // update queue for the subsequent partitions
 									}
@@ -241,15 +246,13 @@ public class LegacyFetcher implements Fetcher {
 
 	private <T> SimpleConsumerThread<T> createAndStartSimpleConsumerThread(SourceFunction.SourceContext<T> sourceContext,
 																		KeyedDeserializationSchema<T> deserializer,
-																		HashMap<KafkaTopicPartition, Long> lastOffsets,
 																		List<FetchPartition> seedPartitions, Node leader) throws IOException, ClassNotFoundException {
-		SimpleConsumerThread<T> brokerThread;
 		final KeyedDeserializationSchema<T> clonedDeserializer =
 				InstantiationUtil.clone(deserializer, userCodeClassloader);
 
 		// seed thread with list of fetch partitions (otherwise it would shut down immediately again
-		brokerThread = new SimpleConsumerThread<>(this, config,
-				leader, seedPartitions, unassignedPartitions, sourceContext, clonedDeserializer, lastOffsets);
+		SimpleConsumerThread<T> brokerThread = new SimpleConsumerThread<>(this, config,
+				leader, seedPartitions, unassignedPartitions, sourceContext, clonedDeserializer);
 
 		brokerThread.setName(String.format("SimpleConsumer - %s - broker-%s (%s:%d)",
 				taskName, leader.id(), leader.host(), leader.port()));
@@ -387,8 +390,7 @@ public class LegacyFetcher implements Fetcher {
 		
 		private final SourceFunction.SourceContext<T> sourceContext;
 		private final KeyedDeserializationSchema<T> deserializer;
-		private final HashMap<KafkaTopicPartition, Long> offsetsState;
-		
+
 		private final List<FetchPartition> partitions;
 		
 		private final Node broker;
@@ -399,8 +401,6 @@ public class LegacyFetcher implements Fetcher {
 
 		private final ClosableBlockingQueue<FetchPartition> unassignedPartitions;
 
-
-
 		private volatile boolean running = true;
 
 		/** Queue containing new fetch partitions for the consumer thread */
@@ -424,15 +424,13 @@ public class LegacyFetcher implements Fetcher {
 									List<FetchPartition> seedPartitions,
 									ClosableBlockingQueue<FetchPartition> unassignedPartitions,
 									SourceFunction.SourceContext<T> sourceContext,
-									KeyedDeserializationSchema<T> deserializer,
-									HashMap<KafkaTopicPartition, Long> offsetsState) {
+									KeyedDeserializationSchema<T> deserializer) {
 			this.owner = owner;
 			this.config = config;
 			this.broker = broker;
 			this.partitions = seedPartitions;
 			this.sourceContext = requireNonNull(sourceContext);
 			this.deserializer = requireNonNull(deserializer);
-			this.offsetsState = requireNonNull(offsetsState);
 			this.unassignedPartitions = requireNonNull(unassignedPartitions);
 
 			this.soTimeout = getIntFromConfig(config, "socket.timeout.ms", 30000);
@@ -661,8 +659,7 @@ public class LegacyFetcher implements Fetcher {
 									continue partitionsLoop;
 								}
 								synchronized (sourceContext.getCheckpointLock()) {
-									sourceContext.collect(value);
-									offsetsState.put(topicPartition, offset);
+									owner.flinkKafkaConsumer.processElement(sourceContext, topicPartition, value, offset);
 								}
 								
 								// advance offset for the next request
@@ -703,7 +700,7 @@ public class LegacyFetcher implements Fetcher {
 			List<FetchPartition> partitionsToGetOffsetsFor = new ArrayList<>();
 
 			for (FetchPartition fp : partitions) {
-				if (fp.nextOffsetToRead == FlinkKafkaConsumer08.OFFSET_NOT_SET) {
+				if (fp.nextOffsetToRead == FlinkKafkaConsumerBase.OFFSET_NOT_SET) {
 					// retrieve the offset from the consumer
 					partitionsToGetOffsetsFor.add(fp);
 				}
@@ -716,7 +713,7 @@ public class LegacyFetcher implements Fetcher {
 				// we subtract -1 from the offset
 				synchronized (sourceContext.getCheckpointLock()) {
 					for(FetchPartition fp: partitionsToGetOffsetsFor) {
-						this.offsetsState.put(new KafkaTopicPartition(fp.topic, fp.partition), fp.nextOffsetToRead - 1L);
+						owner.flinkKafkaConsumer.updateOffsetForPartition(new KafkaTopicPartition(fp.topic, fp.partition), fp.nextOffsetToRead - 1L);
 					}
 				}
 			}

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
index 003e24f..328cab0 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
@@ -24,7 +24,7 @@ import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 
 import org.slf4j.Logger;
@@ -43,7 +43,7 @@ public class ZookeeperOffsetHandler implements OffsetHandler {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class);
 	
-	private static final long OFFSET_NOT_SET = FlinkKafkaConsumer08.OFFSET_NOT_SET;
+	private static final long OFFSET_NOT_SET = FlinkKafkaConsumerBase.OFFSET_NOT_SET;
 
 	private final String groupId;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
index d704fbd..d6ee968 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
@@ -61,6 +61,16 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 	}
 
 	@Test(timeout = 60000)
+	public void testPunctuatedExplicitWMConsumer() throws Exception {
+		runExplicitPunctuatedWMgeneratingConsumerTest(false);
+	}
+
+	@Test(timeout = 60000)
+	public void testPunctuatedExplicitWMConsumerWithEmptyTopic() throws Exception {
+		runExplicitPunctuatedWMgeneratingConsumerTest(true);
+	}
+
+	@Test(timeout = 60000)
 	public void testKeyValueSupport() throws Exception {
 		runKeyValueTest();
 	}
@@ -198,9 +208,9 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 
 		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
 
-		assertTrue(o1 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
-		assertTrue(o2 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o2 >= 0 && o2 <= 100));
-		assertTrue(o3 == FlinkKafkaConsumer08.OFFSET_NOT_SET || (o3 >= 0 && o3 <= 100));
+		assertTrue(o1 == FlinkKafkaConsumerBase.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
+		assertTrue(o2 == FlinkKafkaConsumerBase.OFFSET_NOT_SET || (o2 >= 0 && o2 <= 100));
+		assertTrue(o3 == FlinkKafkaConsumerBase.OFFSET_NOT_SET || (o3 >= 0 && o3 <= 100));
 
 		LOG.info("Manipulating offsets");
 

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
index 113ad71..7337f65 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.commons.collections.map.LinkedMap;
 
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaPartitionState;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
@@ -81,7 +82,7 @@ public class KafkaConsumerTest {
 	@Test
 	public void testSnapshot() {
 		try {
-			Field offsetsField = FlinkKafkaConsumerBase.class.getDeclaredField("offsetsState");
+			Field offsetsField = FlinkKafkaConsumerBase.class.getDeclaredField("partitionState");
 			Field runningField = FlinkKafkaConsumerBase.class.getDeclaredField("running");
 			Field mapField = FlinkKafkaConsumerBase.class.getDeclaredField("pendingCheckpoints");
 
@@ -92,18 +93,19 @@ public class KafkaConsumerTest {
 			FlinkKafkaConsumer08<?> consumer = mock(FlinkKafkaConsumer08.class);
 			when(consumer.snapshotState(anyLong(), anyLong())).thenCallRealMethod();
 
-
+			HashMap<KafkaTopicPartition, KafkaPartitionState> testState = new HashMap<>();
 			HashMap<KafkaTopicPartition, Long> testOffsets = new HashMap<>();
 			long[] offsets = new long[] { 43, 6146, 133, 16, 162, 616 };
 			int j = 0;
 			for (long i: offsets) {
 				KafkaTopicPartition ktp = new KafkaTopicPartition("topic", j++);
+				testState.put(ktp, new KafkaPartitionState(ktp.getPartition(), i));
 				testOffsets.put(ktp, i);
 			}
 
 			LinkedMap map = new LinkedMap();
 
-			offsetsField.set(consumer, testOffsets);
+			offsetsField.set(consumer, testState);
 			runningField.set(consumer, true);
 			mapField.set(consumer, map);
 
@@ -118,7 +120,9 @@ public class KafkaConsumerTest {
 				HashMap<KafkaTopicPartition, Long> checkpointCopy = (HashMap<KafkaTopicPartition, Long>) checkpoint.clone();
 
 				for (Map.Entry<KafkaTopicPartition, Long> e: testOffsets.entrySet()) {
-					testOffsets.put(e.getKey(), e.getValue() + 1);
+					KafkaTopicPartition ktp = e.getKey();
+					testState.put(ktp, new KafkaPartitionState(ktp.getPartition(), e.getValue() + 1));
+					testOffsets.put(ktp, e.getValue() + 1);
 				}
 
 				assertEquals(checkpointCopy, checkpoint);

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
index 3b780bd..55f9875 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
@@ -269,7 +269,7 @@ public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 		this.subscribedPartitionsAsFlink = assignPartitions(this.partitionInfos, numConsumers, thisConsumerIndex);
 		if(this.subscribedPartitionsAsFlink.isEmpty()) {
 			LOG.info("This consumer doesn't have any partitions assigned");
-			this.offsetsState = null;
+			this.partitionState = null;
 			return;
 		} else {
 			StreamingRuntimeContext streamingRuntimeContext = (StreamingRuntimeContext) getRuntimeContext();
@@ -302,13 +302,13 @@ public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 		// check if we need to explicitly seek to a specific offset (restore case)
 		if(restoreToOffset != null) {
 			// we are in a recovery scenario
-			for(Map.Entry<KafkaTopicPartition, Long> offset: restoreToOffset.entrySet()) {
+			for(Map.Entry<KafkaTopicPartition, Long> info: restoreToOffset.entrySet()) {
 				// seek all offsets to the right position
-				this.consumer.seek(new TopicPartition(offset.getKey().getTopic(), offset.getKey().getPartition()), offset.getValue() + 1);
+				this.consumer.seek(new TopicPartition(info.getKey().getTopic(), info.getKey().getPartition()), info.getValue() + 1);
 			}
-			this.offsetsState = restoreToOffset;
+			this.partitionState = restoreInfoFromCheckpoint();
 		} else {
-			this.offsetsState = new HashMap<>();
+			this.partitionState = new HashMap<>();
 		}
 	}
 
@@ -474,8 +474,7 @@ public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 								break pollLoop;
 							}
 							synchronized (sourceContext.getCheckpointLock()) {
-								sourceContext.collect(value);
-								flinkKafkaConsumer.offsetsState.put(flinkPartition, record.offset());
+								flinkKafkaConsumer.processElement(sourceContext, flinkPartition, value, record.offset());
 							}
 						}
 					}

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
index eb152a2..82e1dce 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java
@@ -36,13 +36,22 @@ public class Kafka09ITCase extends KafkaConsumerTestBase {
 		runFailOnNoBrokerTest();
 	}
 
-
 	@Test(timeout = 60000)
 	public void testConcurrentProducerConsumerTopology() throws Exception {
 		runSimpleConcurrentProducerConsumerTopology();
 	}
 
 	@Test(timeout = 60000)
+	public void testPunctuatedExplicitWMConsumer() throws Exception {
+		runExplicitPunctuatedWMgeneratingConsumerTest(false);
+	}
+
+	@Test(timeout = 60000)
+	public void testPunctuatedExplicitWMConsumerWithEmptyTopic() throws Exception {
+		runExplicitPunctuatedWMgeneratingConsumerTest(true);
+	}
+
+	@Test(timeout = 60000)
 	public void testKeyValueSupport() throws Exception {
 		runKeyValueTest();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
index 5f20f16..d9e813f 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
@@ -18,12 +18,22 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.commons.collections.map.LinkedMap;
+import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.TimestampAssigner;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaPartitionState;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,7 +48,7 @@ import static java.util.Objects.requireNonNull;
 import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.checkArgument;
 
 public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFunction<T>
-		implements CheckpointListener, CheckpointedAsynchronously<HashMap<KafkaTopicPartition, Long>>, ResultTypeQueryable<T> {
+		implements CheckpointListener, CheckpointedAsynchronously<HashMap<KafkaTopicPartition, Long>>, ResultTypeQueryable<T>, Triggerable {
 
 	// ------------------------------------------------------------------------
 
@@ -46,6 +56,10 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 
 	private static final long serialVersionUID = -6272159445203409112L;
 
+	/** Magic number to define an unset offset. Negative offsets are not used by Kafka (invalid),
+	 * and we pick a number that is probably (hopefully) not used by Kafka as a magic number for anything else. */
+	public static final long OFFSET_NOT_SET = -915623761776L;
+
 	/** The maximum number of pending non-committed checkpoints to track, to avoid memory leaks */
 	public static final int MAX_NUM_PENDING_CHECKPOINTS = 100;
 
@@ -58,8 +72,13 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 	/** Data for pending but uncommitted checkpoints */
 	protected final LinkedMap pendingCheckpoints = new LinkedMap();
 
-	/** The offsets of the last returned elements */
-	protected transient HashMap<KafkaTopicPartition, Long> offsetsState;
+	/**
+	 * Information about the partitions being read by the local consumer. This contains:
+	 * offsets of the last returned elements, and if a timestamp assigner is used, it
+	 * also contains the maximum seen timestamp in the partition and if the partition
+	 * still receives elements or it is inactive.
+	 */
+	protected transient HashMap<KafkaTopicPartition, KafkaPartitionState> partitionState;
 
 	/** The offsets to restore to, if the consumer restores state from a checkpoint */
 	protected transient HashMap<KafkaTopicPartition, Long> restoreToOffset;
@@ -68,13 +87,41 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 	protected volatile boolean running = true;
 
 	// ------------------------------------------------------------------------
+	//							WATERMARK EMISSION
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The user-specified methods to extract the timestamps from the records in Kafka, and
+	 * to decide when to emit watermarks.
+	 */
+	private AssignerWithPunctuatedWatermarks<T> punctuatedWatermarkAssigner;
+
+	/**
+	 * The user-specified methods to extract the timestamps from the records in Kafka, and
+	 * to decide when to emit watermarks.
+	 */
+	private AssignerWithPeriodicWatermarks<T> periodicWatermarkAssigner;
+
+	private StreamingRuntimeContext runtime = null;
+
+	private SourceContext<T> srcContext = null;
+
+	/**
+	 * The interval between consecutive periodic watermark emissions,
+	 * as configured via the {@link ExecutionConfig#getAutoWatermarkInterval()}.
+	 */
+	private long watermarkInterval = -1;
 
+	/** The last emitted watermark. */
+	private long lastEmittedWatermark = Long.MIN_VALUE;
+
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Creates a new Flink Kafka Consumer, using the given type of fetcher and offset handler.
 	 *
 	 * <p>To determine which kink of fetcher and offset handler to use, please refer to the docs
-	 * at the beginnign of this class.</p>
+	 * at the beginning of this class.</p>
 	 *
 	 * @param deserializer
 	 *           The deserializer to turn raw byte messages into Java/Scala objects.
@@ -85,13 +132,300 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 		this.deserializer = requireNonNull(deserializer, "valueDeserializer");
 	}
 
+	/**
+	 * Specifies an {@link AssignerWithPunctuatedWatermarks} to emit watermarks in a punctuated manner. Bare in mind
+	 * that the source can either have an {@link AssignerWithPunctuatedWatermarks} or an
+	 * {@link AssignerWithPeriodicWatermarks}, not both.
+	 */
+	public FlinkKafkaConsumerBase<T> setPunctuatedWatermarkEmitter(AssignerWithPunctuatedWatermarks<T> assigner) {
+		checkEmitterDuringInit();
+		this.punctuatedWatermarkAssigner = assigner;
+		return this;
+	}
+
+	/**
+	 * Specifies an {@link AssignerWithPeriodicWatermarks} to emit watermarks periodically. Bare in mind that the
+	 * source can either have an {@link AssignerWithPunctuatedWatermarks} or an
+	 * {@link AssignerWithPeriodicWatermarks}, not both.
+	 */
+	public FlinkKafkaConsumerBase<T> setPeriodicWatermarkEmitter(AssignerWithPeriodicWatermarks<T> assigner) {
+		checkEmitterDuringInit();
+		this.periodicWatermarkAssigner = assigner;
+		return this;
+	}
+
+	/**
+	 * Processes the element after having been read from Kafka and deserialized, and updates the
+	 * last read offset for the specifies partition. These two actions should be performed in
+	 * an atomic way in order to guarantee exactly once semantics.
+	 * @param sourceContext
+	 *           The context the task operates in.
+	 * @param partDescriptor
+	 *            A descriptor containing the topic and the id of the partition.
+	 * @param value
+	 *           The element to process.
+	 * @param offset
+	 *           The offset of the element in the partition.
+	 * */
+	public void processElement(SourceContext<T> sourceContext, KafkaTopicPartition partDescriptor, T value, long offset) {
+		if (punctuatedWatermarkAssigner == null && periodicWatermarkAssigner == null) {
+			// the case where no watermark emitter is specified.
+			sourceContext.collect(value);
+		} else {
+
+			if (srcContext == null) {
+				srcContext = sourceContext;
+			}
+
+			long extractedTimestamp = extractTimestampAndEmitElement(partDescriptor, value);
+
+			// depending on the specified watermark emitter, either send a punctuated watermark,
+			// or set the timer for the first periodic watermark. In the periodic case, we set the timer
+			// only for the first watermark, as it is the trigger() that will set the subsequent ones.
+
+			if (punctuatedWatermarkAssigner != null) {
+				final Watermark nextWatermark = punctuatedWatermarkAssigner
+					.checkAndGetNextWatermark(value, extractedTimestamp);
+				if (nextWatermark != null) {
+					emitWatermarkIfMarkingProgress(sourceContext);
+				}
+			} else if(periodicWatermarkAssigner != null && runtime == null) {
+				runtime = (StreamingRuntimeContext) getRuntimeContext();
+				watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval();
+				if (watermarkInterval > 0) {
+					runtime.registerTimer(System.currentTimeMillis() + watermarkInterval, this);
+				}
+			}
+		}
+		updateOffsetForPartition(partDescriptor, offset);
+	}
+
+	/**
+	 * Extract the timestamp from the element based on the user-specified extractor,
+	 * emit the element with the new timestamp, and update the partition monitoring info (if necessary).
+	 * In more detail, upon reception of an element with a timestamp greater than the greatest timestamp
+	 * seen so far in that partition, this method updates the maximum timestamp seen for that partition,
+	 * and marks the partition as {@code active}, i.e. it still receives fresh data.
+	 * @param partDescriptor the partition the new element belongs to.
+	 * @param value the element to be forwarded.
+	 * @return the timestamp of the new element.
+	 */
+	private long extractTimestampAndEmitElement(KafkaTopicPartition partDescriptor, T value) {
+		long extractedTimestamp = getTimestampAssigner().extractTimestamp(value, Long.MIN_VALUE);
+		srcContext.collectWithTimestamp(value, extractedTimestamp);
+		updateMaximumTimestampForPartition(partDescriptor, extractedTimestamp);
+		return extractedTimestamp;
+	}
+
+	/**
+	 * Upon reception of an element with a timestamp greater than the greatest timestamp seen so far in the partition,
+	 * this method updates the maximum timestamp seen for that partition to {@code timestamp}, and marks the partition
+	 * as {@code active}, i.e. it still receives fresh data. If the partition is not known to the system, then a new
+	 * {@link KafkaPartitionState} is created and is associated to the new partition for future monitoring.
+	 * @param partDescriptor
+	 *            A descriptor containing the topic and the id of the partition.
+	 * @param timestamp
+	 *           The timestamp to set the minimum to, if smaller than the already existing one.
+	 * @return {@code true} if the minimum was updated successfully to {@code timestamp}, {@code false}
+	 *           if the previous value is smaller than the provided timestamp
+	 * */
+	private boolean updateMaximumTimestampForPartition(KafkaTopicPartition partDescriptor, long timestamp) {
+		KafkaPartitionState info = getOrInitializeInfo(partDescriptor);
+
+		if(timestamp > info.getMaxTimestamp()) {
+
+			// the flag is set to false as soon as the current partition's max timestamp is sent as a watermark.
+			// if then, and for that partition, only late elements arrive, then the max timestamp will stay the
+			// same, and it will keep the overall system from progressing.
+			// To avoid this, we only mark a partition as active on non-late elements.
+
+			info.setActive(true);
+			info.setMaxTimestamp(timestamp);
+			return  true;
+		}
+		return false;
+	}
+
+	/**
+	 * Updates the last read offset for the partition specified by the {@code partDescriptor} to {@code offset}.
+	 * If it is the first time we see the partition, then a new {@link KafkaPartitionState} is created to monitor
+	 * this specific partition.
+	 * @param partDescriptor the partition whose info to update.
+	 * @param offset the last read offset of the partition.
+	 */
+	public void updateOffsetForPartition(KafkaTopicPartition partDescriptor, long offset) {
+		KafkaPartitionState info = getOrInitializeInfo(partDescriptor);
+		info.setOffset(offset);
+	}
+
+	@Override
+	public void trigger(long timestamp) throws Exception {
+		if(this.srcContext == null) {
+			// if the trigger is called before any elements, then we
+			// just set the next timer to fire when it should and we
+			// ignore the triggering as this would produce no results.
+			setNextWatermarkTimer();
+			return;
+		}
+
+		// this is valid because this method is only called when watermarks
+		// are set to be emitted periodically.
+		final Watermark nextWatermark = periodicWatermarkAssigner.getCurrentWatermark();
+		if(nextWatermark != null) {
+			emitWatermarkIfMarkingProgress(srcContext);
+		}
+		setNextWatermarkTimer();
+	}
+
+	/**
+	 * Emits a new watermark, with timestamp equal to the minimum across all the maximum timestamps
+	 * seen per local partition (across all topics). The new watermark is emitted if and only if
+	 * it signals progress in event-time, i.e. if its timestamp is greater than the timestamp of
+	 * the last emitted watermark. In addition, this method marks as inactive the partition whose
+	 * timestamp was emitted as watermark, i.e. the one with the minimum across the maximum timestamps
+	 * of the local partitions. This is done to avoid not making progress because
+	 * a partition stopped receiving data. The partition is going to be marked as {@code active}
+	 * as soon as the <i>next non-late</i> element arrives.
+	 *
+	 * @return {@code true} if the Watermark was successfully emitted, {@code false} otherwise.
+	 */
+	private boolean emitWatermarkIfMarkingProgress(SourceFunction.SourceContext<T> sourceContext) {
+		Tuple2<KafkaTopicPartition, Long> globalMinTs = getMinTimestampAcrossAllTopics();
+		if(globalMinTs.f0 != null ) {
+			synchronized (sourceContext.getCheckpointLock()) {
+				long minTs = globalMinTs.f1;
+				if(minTs > lastEmittedWatermark) {
+					lastEmittedWatermark = minTs;
+					Watermark toEmit = new Watermark(minTs);
+					sourceContext.emitWatermark(toEmit);
+					return true;
+				}
+			}
+		}
+		return false;
+	}
+
+	/**
+	 * Kafka sources with timestamp extractors are expected to keep the maximum timestamp seen per
+	 * partition they are reading from. This is to mark the per-partition event-time progress.
+	 *
+	 * This method iterates this list, and returns the minimum timestamp across these per-partition
+	 * max timestamps, and across all topics. In addition to this information, it also returns the topic and
+	 * the partition within the topic the timestamp belongs to.
+	 */
+	private Tuple2<KafkaTopicPartition, Long> getMinTimestampAcrossAllTopics() {
+		Tuple2<KafkaTopicPartition, Long> minTimestamp = new Tuple2<>(null, Long.MAX_VALUE);
+		for(Map.Entry<KafkaTopicPartition, KafkaPartitionState> entries: partitionState.entrySet()) {
+			KafkaTopicPartition part = entries.getKey();
+			KafkaPartitionState info = entries.getValue();
+
+			if(partitionIsActive(part) && info.getMaxTimestamp() < minTimestamp.f1) {
+				minTimestamp.f0 = part;
+				minTimestamp.f1 = info.getMaxTimestamp();
+			}
+		}
+
+		if(minTimestamp.f0 != null) {
+			// it means that we have a winner and we have to set its flag to
+			// inactive, until its next non-late element.
+			KafkaTopicPartition partitionDescriptor = minTimestamp.f0;
+			setActiveFlagForPartition(partitionDescriptor, false);
+		}
+
+		return minTimestamp;
+	}
+
+	/**
+	 * Sets the {@code active} flag for a given partition of a topic to {@code isActive}.
+	 * This flag signals if the partition is still receiving data and it is used to avoid the case
+	 * where a partition stops receiving data, so its max seen timestamp does not advance, and it
+	 * holds back the progress of the watermark for all partitions. Note that if the partition is
+	 * not known to the system, then a new {@link KafkaPartitionState} is created and is associated
+	 * to the new partition for future monitoring.
+	 *
+	 * @param partDescriptor
+	 * 				A descriptor containing the topic and the id of the partition.
+	 * @param isActive
+	 * 				The value {@code true} or {@code false} to set the flag to.
+	 */
+	private void setActiveFlagForPartition(KafkaTopicPartition partDescriptor, boolean isActive) {
+		KafkaPartitionState info = getOrInitializeInfo(partDescriptor);
+		info.setActive(isActive);
+	}
+
+	/**
+	 * Gets the statistics for a given partition specified by the {@code partition} argument.
+	 * If it is the first time we see this partition, a new {@link KafkaPartitionState} data structure
+	 * is initialized to monitor it from now on. This method never throws a {@link NullPointerException}.
+	 * @param partition the partition to be fetched.
+	 * @return the gathered statistics for that partition.
+	 * */
+	private KafkaPartitionState getOrInitializeInfo(KafkaTopicPartition partition) {
+		KafkaPartitionState info = partitionState.get(partition);
+		if(info == null) {
+			info = new KafkaPartitionState(partition.getPartition(), FlinkKafkaConsumerBase.OFFSET_NOT_SET);
+			partitionState.put(partition, info);
+		}
+		return info;
+	}
+
+	/**
+	 * Checks if a partition of a topic is still active, i.e. if it still receives data.
+	 * @param partDescriptor
+	 *          A descriptor containing the topic and the id of the partition.
+	 * */
+	private boolean partitionIsActive(KafkaTopicPartition partDescriptor) {
+		KafkaPartitionState info = partitionState.get(partDescriptor);
+		if(info == null) {
+			throw new RuntimeException("Unknown Partition: Topic=" + partDescriptor.getTopic() +
+				" Partition=" + partDescriptor.getPartition());
+		}
+		return info.isActive();
+	}
+
+	private TimestampAssigner<T> getTimestampAssigner() {
+		checkEmitterStateAfterInit();
+		return periodicWatermarkAssigner != null ? periodicWatermarkAssigner : punctuatedWatermarkAssigner;
+	}
+
+	private void setNextWatermarkTimer() {
+		long timeToNextWatermark = System.currentTimeMillis() + watermarkInterval;
+		runtime.registerTimer(timeToNextWatermark, this);
+	}
+
+	private void checkEmitterDuringInit() {
+		if(periodicWatermarkAssigner != null) {
+			throw new RuntimeException("A periodic watermark emitter has already been provided.");
+		} else if(punctuatedWatermarkAssigner != null) {
+			throw new RuntimeException("A punctuated watermark emitter has already been provided.");
+		}
+	}
+
+	private void checkEmitterStateAfterInit() {
+		if(periodicWatermarkAssigner == null && punctuatedWatermarkAssigner == null) {
+			throw new RuntimeException("The timestamp assigner has not been initialized.");
+		} else if(periodicWatermarkAssigner != null && punctuatedWatermarkAssigner != null) {
+			throw new RuntimeException("The source can either have an assigner with punctuated " +
+				"watermarks or one with periodic watermarks, not both.");
+		}
+	}
+
 	// ------------------------------------------------------------------------
 	//  Checkpoint and restore
 	// ------------------------------------------------------------------------
 
+	HashMap<KafkaTopicPartition, KafkaPartitionState> restoreInfoFromCheckpoint() {
+		HashMap<KafkaTopicPartition, KafkaPartitionState> partInfo = new HashMap<>(restoreToOffset.size());
+		for(Map.Entry<KafkaTopicPartition, Long> offsets: restoreToOffset.entrySet()) {
+			KafkaTopicPartition key = offsets.getKey();
+			partInfo.put(key, new KafkaPartitionState(key.getPartition(), offsets.getValue()));
+		}
+		return partInfo;
+	}
+
 	@Override
 	public HashMap<KafkaTopicPartition, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
-		if (offsetsState == null) {
+		if (partitionState == null) {
 			LOG.debug("snapshotState() requested on not yet opened source; returning null.");
 			return null;
 		}
@@ -100,15 +434,16 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 			return null;
 		}
 
+		HashMap<KafkaTopicPartition, Long> currentOffsets = new HashMap<>();
+		for (Map.Entry<KafkaTopicPartition, KafkaPartitionState> entry: partitionState.entrySet()) {
+			currentOffsets.put(entry.getKey(), entry.getValue().getOffset());
+		}
+
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("Snapshotting state. Offsets: {}, checkpoint id: {}, timestamp: {}",
-					KafkaTopicPartition.toString(offsetsState), checkpointId, checkpointTimestamp);
+					KafkaTopicPartition.toString(currentOffsets), checkpointId, checkpointTimestamp);
 		}
 
-		// the use of clone() is okay here is okay, we just need a new map, the keys are not changed
-		//noinspection unchecked
-		HashMap<KafkaTopicPartition, Long> currentOffsets = (HashMap<KafkaTopicPartition, Long>) offsetsState.clone();
-
 		// the map cannot be asynchronously updated, because only one checkpoint call can happen
 		// on this function at a time: either snapshotState() or notifyCheckpointComplete()
 		pendingCheckpoints.put(checkpointId, currentOffsets);
@@ -128,7 +463,7 @@ public abstract class FlinkKafkaConsumerBase<T> extends RichParallelSourceFuncti
 
 	@Override
 	public void notifyCheckpointComplete(long checkpointId) throws Exception {
-		if (offsetsState == null) {
+		if (partitionState == null) {
 			LOG.debug("notifyCheckpointComplete() called on uninitialized source");
 			return;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java
new file mode 100644
index 0000000..11a392a
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaPartitionState.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import java.io.Serializable;
+
+public class KafkaPartitionState implements Serializable {
+
+	private static final long serialVersionUID = 722083576322742328L;
+
+	private final int partitionID;
+	private long offset;
+
+	private long maxTimestamp = Long.MIN_VALUE;
+	private boolean isActive = false;
+
+	public KafkaPartitionState(int id, long offset) {
+		this.partitionID = id;
+		this.offset = offset;
+	}
+
+	public void setOffset(long offset) {
+		this.offset = offset;
+	}
+
+	public void setActive(boolean isActive) {
+		this.isActive = isActive;
+	}
+
+	public void setMaxTimestamp(long timestamp) {
+		maxTimestamp = timestamp;
+	}
+
+	public int getPartition() {
+		return partitionID;
+	}
+
+	public boolean isActive() {
+		return isActive;
+	}
+
+	public long getMaxTimestamp() {
+		return maxTimestamp;
+	}
+
+	public long getOffset() {
+		return offset;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index 2cd59e6..340950b 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -32,6 +32,7 @@ import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
@@ -50,15 +51,21 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
 import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators;
 import org.apache.flink.streaming.connectors.kafka.testutils.DiscardingSink;
@@ -69,6 +76,7 @@ import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidating
 import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2Partitioner;
 import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
@@ -374,7 +382,6 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		deleteTestTopic(topic);
 	}
 
-
 	/**
 	 * Tests the proper consumption when having a 1:1 correspondence between kafka partitions and
 	 * Flink sources.
@@ -1443,4 +1450,229 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			this.numElementsTotal = state;
 		}
 	}
+
+	/////////////			Testing the Kafka consumer with embeded watermark generation functionality			///////////////
+
+	@RetryOnException(times=0, exception=kafka.common.NotLeaderForPartitionException.class)
+	public void runExplicitPunctuatedWMgeneratingConsumerTest(boolean emptyPartition) throws Exception {
+
+		final String topic1 = "wmExtractorTopic1_" + UUID.randomUUID().toString();
+		final String topic2 = "wmExtractorTopic2_" + UUID.randomUUID().toString();
+
+		final Map<String, Boolean> topics = new HashMap<>();
+		topics.put(topic1, false);
+		topics.put(topic2, emptyPartition);
+
+		final int noOfTopcis = topics.size();
+		final int partitionsPerTopic = 1;
+		final int elementsPerPartition = 100 + 1;
+
+		final int totalElements = emptyPartition ?
+			partitionsPerTopic * elementsPerPartition :
+			noOfTopcis * partitionsPerTopic * elementsPerPartition;
+
+		createTestTopic(topic1, partitionsPerTopic, 1);
+		createTestTopic(topic2, partitionsPerTopic, 1);
+
+		final StreamExecutionEnvironment env =
+			StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.setParallelism(partitionsPerTopic);
+		env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately
+		env.getConfig().disableSysoutLogging();
+
+		TypeInformation<Tuple2<Long, Integer>> longIntType = TypeInfoParser.parse("Tuple2<Long, Integer>");
+
+		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+		producerProperties.setProperty("retries", "0");
+
+		putDataInTopics(env, producerProperties, elementsPerPartition, topics, longIntType);
+
+		List<String> topicTitles = new ArrayList<>(topics.keySet());
+		runPunctuatedComsumer(env, topicTitles, totalElements, longIntType);
+
+		executeAndCatchException(env, "runComsumerWithPunctuatedExplicitWMTest");
+
+		for(String topic: topicTitles) {
+			deleteTestTopic(topic);
+		}
+	}
+
+	private void executeAndCatchException(StreamExecutionEnvironment env, String execName) throws Exception {
+		try {
+			tryExecutePropagateExceptions(env, execName);
+		}
+		catch (ProgramInvocationException | JobExecutionException e) {
+			// look for NotLeaderForPartitionException
+			Throwable cause = e.getCause();
+
+			// search for nested SuccessExceptions
+			int depth = 0;
+			while (cause != null && depth++ < 20) {
+				if (cause instanceof kafka.common.NotLeaderForPartitionException) {
+					throw (Exception) cause;
+				}
+				cause = cause.getCause();
+			}
+			throw e;
+		}
+	}
+
+	private void putDataInTopics(StreamExecutionEnvironment env,
+								Properties producerProperties,
+								final int elementsPerPartition,
+								Map<String, Boolean> topics,
+								TypeInformation<Tuple2<Long, Integer>> outputTypeInfo) {
+		if(topics.size() != 2) {
+			throw new RuntimeException("This method accepts two topics as arguments.");
+		}
+
+		TypeInformationSerializationSchema<Tuple2<Long, Integer>> sinkSchema =
+			new TypeInformationSerializationSchema<>(outputTypeInfo, env.getConfig());
+
+		DataStream<Tuple2<Long, Integer>> stream = env
+			.addSource(new RichParallelSourceFunction<Tuple2<Long, Integer>>() {
+				private boolean running = true;
+
+				@Override
+				public void run(SourceContext<Tuple2<Long, Integer>> ctx) throws InterruptedException {
+					int topic = 0;
+					int currentTs = 1;
+
+					while (running && currentTs < elementsPerPartition) {
+						long timestamp = (currentTs % 10 == 0) ? -1L : currentTs;
+						ctx.collect(new Tuple2<Long, Integer>(timestamp, topic));
+						currentTs++;
+					}
+
+					Tuple2<Long, Integer> toWrite2 = new Tuple2<Long, Integer>(-1L, topic);
+					ctx.collect(toWrite2);
+				}
+
+				@Override
+				public void cancel() {
+				running = false;
+			}
+			}).setParallelism(1);
+
+		List<Map.Entry<String, Boolean>> topicsL = new ArrayList<>(topics.entrySet());
+		stream.map(new MapFunction<Tuple2<Long,Integer>, Tuple2<Long,Integer>>() {
+
+			@Override
+			public Tuple2<Long, Integer> map(Tuple2<Long, Integer> value) throws Exception {
+				return value;
+			}
+		}).setParallelism(1).addSink(kafkaServer.getProducer(topicsL.get(0).getKey(),
+			new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null)).setParallelism(1);
+
+		if(!topicsL.get(1).getValue()) {
+			stream.map(new MapFunction<Tuple2<Long,Integer>, Tuple2<Long,Integer>>() {
+
+				@Override
+				public Tuple2<Long, Integer> map(Tuple2<Long, Integer> value) throws Exception {
+					long timestamp = (value.f0 == -1) ? -1L : 1000 + value.f0;
+					return new Tuple2<Long, Integer>(timestamp, 1);
+				}
+			}).setParallelism(1).addSink(kafkaServer.getProducer(topicsL.get(1).getKey(),
+				new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null)).setParallelism(1);
+		}
+	}
+
+	private DataStreamSink<Tuple2<Long, Integer>> runPunctuatedComsumer(StreamExecutionEnvironment env,
+																		List<String> topics,
+																		final int totalElementsToExpect,
+																		TypeInformation<Tuple2<Long, Integer>> inputTypeInfo) {
+
+		TypeInformationSerializationSchema<Tuple2<Long, Integer>> sourceSchema =
+			new TypeInformationSerializationSchema<>(inputTypeInfo, env.getConfig());
+
+		FlinkKafkaConsumerBase<Tuple2<Long, Integer>> source = kafkaServer
+			.getConsumer(topics, sourceSchema, standardProps)
+			.setPunctuatedWatermarkEmitter(new TestPunctuatedTSExtractor());
+
+		DataStreamSource<Tuple2<Long, Integer>> consuming = env.setParallelism(1).addSource(source);
+
+		return consuming
+			.transform("testingWatermarkOperator", inputTypeInfo, new WMTestingOperator())
+			.addSink(new RichSinkFunction<Tuple2<Long, Integer>>() {
+
+				private int elementCount = 0;
+
+				@Override
+				public void invoke(Tuple2<Long, Integer> value) throws Exception {
+					elementCount++;
+					if (elementCount == totalElementsToExpect) {
+						throw new SuccessException();
+					}
+				}
+
+				@Override
+				public void close() throws Exception {
+					super.close();
+				}
+			});
+	}
+
+	/** An extractor that emits a Watermark whenever the timestamp <b>in the record</b> is equal to {@code -1}. */
+	private static class TestPunctuatedTSExtractor implements AssignerWithPunctuatedWatermarks<Tuple2<Long, Integer>> {
+
+		@Override
+		public Watermark checkAndGetNextWatermark(Tuple2<Long, Integer> lastElement, long extractedTimestamp) {
+			return (lastElement.f0 == -1) ? new Watermark(extractedTimestamp) : null;
+		}
+
+		@Override
+		public long extractTimestamp(Tuple2<Long, Integer> element, long previousElementTimestamp) {
+			return element.f0;
+		}
+	}
+
+	private static class WMTestingOperator extends AbstractStreamOperator<Tuple2<Long, Integer>> implements OneInputStreamOperator<Tuple2<Long, Integer>, Tuple2<Long, Integer>> {
+
+		private long lastReceivedWatermark = Long.MIN_VALUE;
+
+		private Map<Integer, Boolean> isEligible = new HashMap<>();
+		private Map<Integer, Long> perPartitionMaxTs = new HashMap<>();
+
+		WMTestingOperator() {
+			isEligible = new HashMap<>();
+			perPartitionMaxTs = new HashMap<>();
+		}
+
+		@Override
+		public void processElement(StreamRecord<Tuple2<Long, Integer>> element) throws Exception {
+			int partition = element.getValue().f1;
+			Long maxTs = perPartitionMaxTs.get(partition);
+			if(maxTs == null || maxTs < element.getValue().f0) {
+				perPartitionMaxTs.put(partition, element.getValue().f0);
+				isEligible.put(partition, element.getValue().f0 > lastReceivedWatermark);
+			}
+			output.collect(element);
+		}
+
+		@Override
+		public void processWatermark(Watermark mark) throws Exception {
+			int partition = -1;
+			long minTS = Long.MAX_VALUE;
+			for (Integer part : perPartitionMaxTs.keySet()) {
+				Long ts = perPartitionMaxTs.get(part);
+				if (ts < minTS && isEligible.get(part)) {
+					partition = part;
+					minTS = ts;
+					lastReceivedWatermark = ts;
+				}
+			}
+			isEligible.put(partition, false);
+
+			assertEquals(minTS, mark.getTimestamp());
+			output.emitWatermark(mark);
+		}
+
+		@Override
+		public void close() throws Exception {
+			super.close();
+			perPartitionMaxTs.clear();
+			isEligible.clear();
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0ac1549e/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
index 38ee394..4b17300 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/AssignerWithPeriodicWatermarks.java
@@ -29,7 +29,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
  * 
  * <p>Use this class to generate watermarks in a periodical interval.
  * At most every {@code i} milliseconds (configured via
- * {@link ExecutionConfig#getAutoWatermarkInterval()}, the system will call the
+ * {@link ExecutionConfig#getAutoWatermarkInterval()}), the system will call the
  * {@link #getCurrentWatermark()} method to probe for the next watermark value.
  * The system will generate a new watermark, if the probed value is non-null
  * and has a timestamp larger than that of the previous watermark (to preserve


[07/14] flink git commit: [FLINK-3700] [core] Add 'Preconditions' utility class.

Posted by se...@apache.org.
[FLINK-3700] [core] Add 'Preconditions' utility class.


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

Branch: refs/heads/master
Commit: 885d543be8a8c0d1acdffa0003e394347d5376ef
Parents: 272fd12
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Apr 5 13:23:14 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/util/Preconditions.java    | 213 +++++++++++++++++++
 1 file changed, 213 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/885d543b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/Preconditions.java b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java
new file mode 100644
index 0000000..135038b
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// ----------------------------------------------------------------------------
+//  This class is largely adapted from "com.google.common.base.Preconditions",
+//  which is part of the "Guava" library.
+//
+//  Because of frequent issues with dependency conflicts, this class was
+//  added to the Flink code base to reduce dependency on Guava.
+// ----------------------------------------------------------------------------
+
+package org.apache.flink.util;
+
+import org.apache.flink.annotation.Internal;
+
+import javax.annotation.Nullable;
+
+/**
+ * A collection of static utility methods to validate input.
+ * 
+ * <p>This class is modelled after Google Guava's Preconditions class, and partly takes code
+ * from that class. We add this code to the Flink code base in order to reduce external
+ * dependencies.
+ */
+@Internal
+public final class Preconditions {
+
+	// ------------------------------------------------------------------------
+	//  Null checks
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Ensures that the given object reference is not null.
+	 * Upon violation, a {@code NullPointerException} with no message is thrown.
+	 * 
+	 * @param reference The object reference
+	 * @return The object reference itself (generically typed).
+	 * 
+	 * @throws NullPointerException Thrown, if the passed reference was null.
+	 */
+	public static <T> T checkNotNull(T reference) {
+		if (reference == null) {
+			throw new NullPointerException();
+		}
+		return reference;
+	}
+	
+	/**
+	 * Ensures that the given object reference is not null.
+	 * Upon violation, a {@code NullPointerException} with the given message is thrown.
+	 * 
+	 * @param reference The object reference
+	 * @param errorMessage The message for the {@code NullPointerException} that is thrown if the check fails.
+	 * @return The object reference itself (generically typed).
+	 *
+	 * @throws NullPointerException Thrown, if the passed reference was null.
+	 */
+	public static <T> T checkNotNull(T reference, @Nullable String errorMessage) {
+		if (reference == null) {
+			throw new NullPointerException(String.valueOf(errorMessage));
+		}
+		return reference;
+	}
+
+	/**
+	 * Ensures that the given object reference is not null.
+	 * Upon violation, a {@code NullPointerException} with the given message is thrown.
+	 * 
+	 * <p>The error message is constructed from a template and an arguments array, after
+	 * a similar fashion as {@link String#format(String, Object...)}, but supporting only
+	 * {@code %s} as a placeholder.
+	 *
+	 * @param reference The object reference
+	 * @param errorMessageTemplate The message template for the {@code NullPointerException}
+	 *                             that is thrown if the check fails. The template substitutes its
+	 *                             {@code %s} placeholders with the error message arguments.
+	 * @param errorMessageArgs The arguments for the error message, to be inserted into the
+	 *                         message template for the {@code %s} placeholders.
+	 *                         
+	 * @return The object reference itself (generically typed).
+	 *
+	 * @throws NullPointerException Thrown, if the passed reference was null.
+	 */
+	public static <T> T checkNotNull(T reference,
+				@Nullable String errorMessageTemplate,
+				@Nullable Object... errorMessageArgs) {
+		
+		if (reference == null) {
+			throw new NullPointerException(format(errorMessageTemplate, errorMessageArgs));
+		}
+		return reference;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Boolean Condition Checking
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Checks the given boolean condition, and throws an {@code IllegalArgumentException} if
+	 * the condition is not met (evaluates to {@code false}).
+	 *
+	 * @param condition The condition to check
+	 *                     
+	 * @throws IllegalArgumentException Thrown, if the condition is violated.
+	 */
+	public static void checkArgument(boolean condition) {
+		if (!condition) {
+			throw new IllegalArgumentException();
+		}
+	}
+
+	/**
+	 * Checks the given boolean condition, and throws an {@code IllegalArgumentException} if
+	 * the condition is not met (evaluates to {@code false}). The exception will have the
+	 * given error message.
+	 *
+	 * @param condition The condition to check
+	 * @param errorMessage The message for the {@code IllegalArgumentException} that is thrown if the check fails.
+	 * 
+	 * @throws IllegalArgumentException Thrown, if the condition is violated.
+	 */
+	public static void checkArgument(boolean condition, @Nullable Object errorMessage) {
+		if (!condition) {
+			throw new IllegalArgumentException(String.valueOf(errorMessage));
+		}
+	}
+
+	/**
+	 * Checks the given boolean condition, and throws an {@code IllegalArgumentException} if
+	 * the condition is not met (evaluates to {@code false}).
+	 *
+	 * @param condition The condition to check
+	 * @param errorMessageTemplate The message template for the {@code IllegalArgumentException}
+	 *                             that is thrown if the check fails. The template substitutes its
+	 *                             {@code %s} placeholders with the error message arguments.
+	 * @param errorMessageArgs The arguments for the error message, to be inserted into the
+	 *                         message template for the {@code %s} placeholders.
+	 * 
+	 * @throws IllegalArgumentException Thrown, if the condition is violated.
+	 */
+	public static void checkArgument(boolean condition,
+			@Nullable String errorMessageTemplate,
+			@Nullable Object... errorMessageArgs) {
+		
+		if (!condition) {
+			throw new IllegalArgumentException(format(errorMessageTemplate, errorMessageArgs));
+		}
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A simplified formatting method. Similar to {@link String#format(String, Object...)}, but
+	 * with lower overhead (only String parameters, no locale, no format validation).
+	 * 
+	 * <p>This method is taken quasi verbatim from the Guava Preconditions class.
+	 */
+	private static String format(@Nullable String template, @Nullable Object... args) {
+		final int numArgs = args == null ? 0 : args.length;
+		template = String.valueOf(template); // null -> "null"
+		
+		// start substituting the arguments into the '%s' placeholders
+		StringBuilder builder = new StringBuilder(template.length() + 16 * numArgs);
+		int templateStart = 0;
+		int i = 0;
+		while (i < numArgs) {
+			int placeholderStart = template.indexOf("%s", templateStart);
+			if (placeholderStart == -1) {
+				break;
+			}
+			builder.append(template.substring(templateStart, placeholderStart));
+			builder.append(args[i++]);
+			templateStart = placeholderStart + 2;
+		}
+		builder.append(template.substring(templateStart));
+
+		// if we run out of placeholders, append the extra args in square braces
+		if (i < numArgs) {
+			builder.append(" [");
+			builder.append(args[i++]);
+			while (i < numArgs) {
+				builder.append(", ");
+				builder.append(args[i++]);
+			}
+			builder.append(']');
+		}
+
+		return builder.toString();
+	}
+	
+	// ------------------------------------------------------------------------
+
+	/** Private constructor to prevent instantiation */
+	private Preconditions() {}
+}
\ No newline at end of file


[02/14] flink git commit: [hotfix] [kafka consumer] Increase Kafka test stability by validating written data before consuming

Posted by se...@apache.org.
[hotfix] [kafka consumer] Increase Kafka test stability by validating written data before consuming


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

Branch: refs/heads/master
Commit: d20eda1b5252d888189af29a2b493023b4621a88
Parents: af79988
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Apr 12 20:14:50 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 .../connectors/kafka/Kafka08ITCase.java         |  72 ++----
 .../connectors/kafka/KafkaConsumerTestBase.java | 217 +++++++++++++++----
 .../connectors/kafka/KafkaTestBase.java         |   4 -
 .../testutils/JobManagerCommunicationUtils.java |  24 +-
 .../kafka/testutils/Tuple2Partitioner.java      |   7 +-
 .../org/apache/flink/test/util/TestUtils.java   |   4 +-
 6 files changed, 217 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d20eda1b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
index 0aef3bd..530c032 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
@@ -18,9 +18,7 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.recipes.cache.TreeCache;
-import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
-import org.apache.curator.framework.recipes.cache.TreeCacheListener;
+
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.runtime.client.JobCancellationException;
 import org.apache.flink.streaming.api.datastream.DataStream;
@@ -33,7 +31,6 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
 import static org.junit.Assert.assertEquals;
@@ -91,16 +88,11 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 
 	@Test(timeout = 60000)
 	public void testInvalidOffset() throws Exception {
-		final String topic = "invalidOffsetTopic";
+		
 		final int parallelism = 1;
-
-		// create topic
-		createTestTopic(topic, parallelism, 1);
-
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-
+		
 		// write 20 messages into topic:
-		writeSequence(env, topic, 20, parallelism);
+		final String topic = writeSequence("invalidOffsetTopic", 20, parallelism, 1);
 
 		// set invalid offset:
 		CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
@@ -110,6 +102,10 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		// read from topic
 		final int valuesCount = 20;
 		final int startFrom = 0;
+
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		env.getConfig().disableSysoutLogging();
+		
 		readSequence(env, standardProps, parallelism, topic, valuesCount, startFrom);
 
 		deleteTestTopic(topic);
@@ -193,10 +189,10 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 	 */
 	@Test(timeout = 60000)
 	public void testOffsetInZookeeper() throws Exception {
-		final String topicName = "testOffsetInZK";
 		final int parallelism = 3;
 
-		createTestTopic(topicName, parallelism, 1);
+		// write a sequence from 0 to 99 to each of the 3 partitions.
+		final String topicName = writeSequence("testOffsetInZK", 100, parallelism, 1);
 
 		StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
 		env1.getConfig().disableSysoutLogging();
@@ -210,16 +206,7 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env2.setParallelism(parallelism);
 
-		StreamExecutionEnvironment env3 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env3.getConfig().disableSysoutLogging();
-		env3.enableCheckpointing(50);
-		env3.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env3.setParallelism(parallelism);
-
-		// write a sequence from 0 to 99 to each of the 3 partitions.
-		writeSequence(env1, topicName, 100, parallelism);
-
-		readSequence(env2, standardProps, parallelism, topicName, 100, 0);
+		readSequence(env1, standardProps, parallelism, topicName, 100, 0);
 
 		CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
 
@@ -243,33 +230,23 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		curatorClient.close();
 
 		// create new env
-		readSequence(env3, standardProps, parallelism, topicName, 50, 50);
+		readSequence(env2, standardProps, parallelism, topicName, 50, 50);
 
 		deleteTestTopic(topicName);
 	}
 
 	@Test(timeout = 60000)
 	public void testOffsetAutocommitTest() throws Exception {
-		final String topicName = "testOffsetAutocommit";
 		final int parallelism = 3;
 
-		createTestTopic(topicName, parallelism, 1);
-
-		StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env1.getConfig().disableSysoutLogging();
-		env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env1.setParallelism(parallelism);
-
-		StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		// NOTE: We are not enabling the checkpointing!
-		env2.getConfig().disableSysoutLogging();
-		env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env2.setParallelism(parallelism);
-
-
 		// write a sequence from 0 to 99 to each of the 3 partitions.
-		writeSequence(env1, topicName, 100, parallelism);
+		final String topicName = writeSequence("testOffsetAutocommit", 100, parallelism, 1);
 
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+		// NOTE: We are not enabling the checkpointing!
+		env.getConfig().disableSysoutLogging();
+		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+		env.setParallelism(parallelism);
 
 		// the readSequence operation sleeps for 20 ms between each record.
 		// setting a delay of 25*20 = 500 for the commit interval makes
@@ -280,7 +257,7 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		readProps.setProperty("auto.commit.interval.ms", "500");
 
 		// read so that the offset can be committed to ZK
-		readSequence(env2, readProps, parallelism, topicName, 100, 0);
+		readSequence(env, readProps, parallelism, topicName, 100, 0);
 
 		// get the offset
 		CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
@@ -314,19 +291,10 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 	 */
 	@Test(timeout = 60000)
 	public void testKafkaOffsetRetrievalToZookeeper() throws Exception {
-		final String topicName = "testKafkaOffsetToZk";
 		final int parallelism = 3;
-		
-		createTestTopic(topicName, parallelism, 1);
-		
-		StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env1.getConfig().disableSysoutLogging();
-		env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env1.setParallelism(parallelism);
 
 		// write a sequence from 0 to 49 to each of the 3 partitions.
-		writeSequence(env1, topicName, 50, parallelism);
-
+		final String topicName =  writeSequence("testKafkaOffsetToZk", 50, parallelism, 1);
 
 		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
 		env2.getConfig().disableSysoutLogging();

http://git-wip-us.apache.org/repos/asf/flink/blob/d20eda1b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index dd468a4..8ff67b4 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -35,6 +35,7 @@ import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -49,13 +50,13 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.runtime.client.JobExecutionException;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.state.CheckpointListener;
-import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
@@ -65,7 +66,6 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.kafka.testutils.DataGenerators;
-import org.apache.flink.streaming.connectors.kafka.testutils.DiscardingSink;
 import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.JobManagerCommunicationUtils;
 import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidatingMapper;
@@ -74,6 +74,7 @@ import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2Partitioner;
 import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema;
@@ -90,6 +91,7 @@ import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.errors.TimeoutException;
 import org.junit.Assert;
 
+import org.junit.Before;
 import org.junit.Rule;
 
 import java.io.ByteArrayInputStream;
@@ -121,8 +123,22 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	
 	@Rule
 	public RetryRule retryRule = new RetryRule();
-	
 
+
+	// ------------------------------------------------------------------------
+	//  Common Test Preparation
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Makes sure that no job is on the JobManager any more from any previous tests that use
+	 * the same mini cluster. Otherwise, missing slots may happen.
+	 */
+	@Before
+	public void ensureNoJobIsLingering() throws Exception {
+		JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout));
+	}
+	
+	
 	// ------------------------------------------------------------------------
 	//  Suite of Tests
 	//
@@ -131,7 +147,6 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	//  select which tests to run.
 	// ------------------------------------------------------------------------
 
-
 	/**
 	 * Test that ensures the KafkaConsumer is properly failing if the topic doesnt exist
 	 * and a wrong broker was specified
@@ -1080,17 +1095,9 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 	 * @throws Exception
 	 */
 	public void runMetricsAndEndOfStreamTest() throws Exception {
-		final String topic = "testEndOfStream";
-		createTestTopic(topic, 1, 1);
-		final int ELEMENT_COUNT = 300;
 
-		// write some data
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-		env.setParallelism(1);
-		env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
-		env.getConfig().disableSysoutLogging();
-
-		writeSequence(env, topic, ELEMENT_COUNT, 1);
+		final int ELEMENT_COUNT = 300;
+		final String topic = writeSequence("testEndOfStream", ELEMENT_COUNT, 1, 1);
 
 		// read using custom schema
 		final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
@@ -1098,7 +1105,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env1.getConfig().disableSysoutLogging();
 
-		DataStream<Tuple2<Integer, Integer>> fromKafka = env.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), standardProps));
+		DataStream<Tuple2<Integer, Integer>> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), standardProps));
 		fromKafka.flatMap(new FlatMapFunction<Tuple2<Integer,Integer>, Void>() {
 			@Override
 			public void flatMap(Tuple2<Integer, Integer> value, Collector<Void> out) throws Exception {
@@ -1106,12 +1113,12 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			}
 		});
 
-		JobExecutionResult result = tryExecute(env, "Consume " + ELEMENT_COUNT + " elements from Kafka");
+		JobExecutionResult result = tryExecute(env1, "Consume " + ELEMENT_COUNT + " elements from Kafka");
 
 		Map<String, Object> accuResults = result.getAllAccumulatorResults();
 		// kafka 0.9 consumer: 39 results
-		if(kafkaServer.getVersion().equals("0.9")) {
-			Assert.assertTrue("Not enough accumulators from Kafka Consumer: " + accuResults.size(), accuResults.size() > 38);
+		if (kafkaServer.getVersion().equals("0.9")) {
+			assertTrue("Not enough accumulators from Kafka Consumer: " + accuResults.size(), accuResults.size() > 38);
 		}
 
 		deleteTestTopic(topic);
@@ -1188,7 +1195,8 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 						int v = values[i];
 						if (v != sourceParallelism) {
 							printTopic(topicName, valuesCount, deser);
-							throw new RuntimeException("Expected v to be 3, but was " + v + " on element " + i + " array=" + Arrays.toString(values));
+							throw new RuntimeException("Expected v to be " + sourceParallelism + 
+									", but was " + v + " on element " + i + " array=" + Arrays.toString(values));
 						}
 					}
 					// test has passed
@@ -1203,42 +1211,161 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 		LOG.info("Successfully read sequence for verification");
 	}
 
-	protected void writeSequence(StreamExecutionEnvironment env, String topicName, final int numElements, int parallelism) throws Exception {
+	protected String writeSequence(
+			String baseTopicName,
+			final int numElements,
+			final int parallelism,
+			final int replicationFactor) throws Exception
+	{
+		LOG.info("\n===================================\n" +
+				"== Writing sequence of " + numElements + " into " + baseTopicName + " with p=" + parallelism + "\n" +
+				"===================================");
 
-		LOG.info("\n===================================\n== Writing sequence of "+numElements+" into "+topicName+" with p="+parallelism+"\n===================================");
-		TypeInformation<Tuple2<Integer, Integer>> resultType = TypeInfoParser.parse("Tuple2<Integer, Integer>");
+		final TypeInformation<Tuple2<Integer, Integer>> resultType = 
+				TypeInformation.of(new TypeHint<Tuple2<Integer, Integer>>() {});
 
-		DataStream<Tuple2<Integer, Integer>> stream = env.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
+		final KeyedSerializationSchema<Tuple2<Integer, Integer>> serSchema =
+				new KeyedSerializationSchemaWrapper<>(
+						new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig()));
 
-			private boolean running = true;
+		final KeyedDeserializationSchema<Tuple2<Integer, Integer>> deserSchema =
+				new KeyedDeserializationSchemaWrapper<>(
+						new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig()));
+		
+		final int maxNumAttempts = 10;
 
-			@Override
-			public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
-				int cnt = 0;
-				int partition = getRuntimeContext().getIndexOfThisSubtask();
+		for (int attempt = 1; attempt <= maxNumAttempts; attempt++) {
+			
+			final String topicName = baseTopicName + '-' + attempt;
+			
+			LOG.info("Writing attempt #1");
+			
+			// -------- Write the Sequence --------
+			
+			createTestTopic(topicName, parallelism, replicationFactor);
 
-				while (running && cnt < numElements) {
-					ctx.collect(new Tuple2<>(partition, cnt));
-					cnt++;
+			StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+			writeEnv.getConfig().disableSysoutLogging();
+			
+			DataStream<Tuple2<Integer, Integer>> stream = writeEnv.addSource(new RichParallelSourceFunction<Tuple2<Integer, Integer>>() {
+	
+				private boolean running = true;
+	
+				@Override
+				public void run(SourceContext<Tuple2<Integer, Integer>> ctx) throws Exception {
+					int cnt = 0;
+					int partition = getRuntimeContext().getIndexOfThisSubtask();
+	
+					while (running && cnt < numElements) {
+						ctx.collect(new Tuple2<>(partition, cnt));
+						cnt++;
+					}
 				}
-			}
+	
+				@Override
+				public void cancel() {
+					running = false;
+				}
+			}).setParallelism(parallelism);
+	
+			// the producer must not produce duplicates
+			Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
+			producerProperties.setProperty("retries", "0");
+			
+			stream.addSink(kafkaServer.getProducer(
+							topicName, serSchema, producerProperties,
+							new Tuple2Partitioner(parallelism)))
+					.setParallelism(parallelism);
 
-			@Override
-			public void cancel() {
-				running = false;
-			}
-		}).setParallelism(parallelism);
+			writeEnv.execute("Write sequence");
+			LOG.info("Finished writing sequence");
 
-		Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings);
-		producerProperties.setProperty("retries", "3");
-		stream.addSink(kafkaServer.getProducer(topicName,
-				new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(resultType, env.getConfig())),
-				producerProperties,
-				new Tuple2Partitioner(parallelism))).setParallelism(parallelism);
+			// -------- Validate the Sequence --------
+			
+			// we need to validate the sequence, because kafka's producers are not exactly once
+			LOG.info("Validating sequence");
+			
+			final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
+			readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart());
+			readEnv.getConfig().disableSysoutLogging();
+			readEnv.setParallelism(parallelism);
+			
+			Properties readProps = (Properties) standardProps.clone();
+			readProps.setProperty("group.id", "flink-tests-validator");
+			
+			FlinkKafkaConsumerBase<Tuple2<Integer, Integer>> consumer = kafkaServer.getConsumer(topicName, deserSchema, readProps);
 
-		env.execute("Write sequence");
+			readEnv
+					.addSource(consumer)
+					.map(new RichMapFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>>() {
+						
+						private final int totalCount = parallelism * numElements;
+						private int count = 0;
+						
+						@Override
+						public Tuple2<Integer, Integer> map(Tuple2<Integer, Integer> value) throws Exception {
+							if (++count == totalCount) {
+								throw new SuccessException();
+							} else {
+								return value;
+							}
+						}
+					}).setParallelism(1)
+					.addSink(new DiscardingSink<Tuple2<Integer, Integer>>()).setParallelism(1);
+			
+			final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+			
+			Thread runner = new Thread() {
+				@Override
+				public void run() {
+					try {
+						tryExecute(readEnv, "sequence validation");
+					} catch (Throwable t) {
+						errorRef.set(t);
+					}
+				}
+			};
+			runner.start();
+			
+			final long deadline = System.currentTimeMillis() + 10000;
+			long delay;
+			while (runner.isAlive() && (delay = deadline - System.currentTimeMillis()) > 0) {
+				runner.join(delay);
+			}
+			
+			boolean success;
+			
+			if (runner.isAlive()) {
+				// did not finish in time, maybe the producer dropped one or more records and
+				// the validation did not reach the exit point
+				success = false;
+				JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
+			}
+			else {
+				Throwable error = errorRef.get();
+				if (error != null) {
+					success = false;
+					LOG.info("Attempt " + attempt + " failed with exception", error);
+				}
+				else {
+					success = true;
+				}
+			}
 
-		LOG.info("Finished writing sequence");
+			JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout));
+			
+			if (success) {
+				// everything is good!
+				return topicName;
+			}
+			else {
+				deleteTestTopic(topicName);
+				// fall through the loop
+			}
+		}
+		
+		throw new Exception("Could not write a valid sequence to Kafka after " + maxNumAttempts + " attempts");
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/d20eda1b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
index c475fad..64b9106 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
@@ -17,9 +17,6 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import kafka.consumer.ConsumerConfig;
-
-
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -30,7 +27,6 @@ import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.TestLogger;
 
-
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d20eda1b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
index 24822ed..f2f761d 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/JobManagerCommunicationUtils.java
@@ -31,10 +31,28 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 public class JobManagerCommunicationUtils {
-	
+
 	private static final FiniteDuration askTimeout = new FiniteDuration(30, TimeUnit.SECONDS);
-	
-	
+
+
+	public static void waitUntilNoJobIsRunning(ActorGateway jobManager) throws Exception {
+		while (true) {
+			// find the jobID
+			Future<Object> listResponse = jobManager.ask(
+					JobManagerMessages.getRequestRunningJobsStatus(), askTimeout);
+
+			Object result = Await.result(listResponse, askTimeout);
+			List<JobStatusMessage> jobs = ((JobManagerMessages.RunningJobsStatus) result).getStatusMessages();
+
+
+			if (jobs.isEmpty()) {
+				return;
+			}
+
+			Thread.sleep(50);
+		}
+	}
+
 	public static void cancelCurrentJob(ActorGateway jobManager) throws Exception {
 		JobStatusMessage status = null;
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/d20eda1b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
index 0844412..c9e9ac1 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/Tuple2Partitioner.java
@@ -28,12 +28,11 @@ import java.io.Serializable;
  * and that expects a specific number of partitions.
  */
 public class Tuple2Partitioner extends KafkaPartitioner<Tuple2<Integer, Integer>> implements Serializable {
-	
+
 	private static final long serialVersionUID = 1L;
 
 	private final int expectedPartitions;
 
-	
 	public Tuple2Partitioner(int expectedPartitions) {
 		this.expectedPartitions = expectedPartitions;
 	}
@@ -43,9 +42,7 @@ public class Tuple2Partitioner extends KafkaPartitioner<Tuple2<Integer, Integer>
 		if (numPartitions != expectedPartitions) {
 			throw new IllegalArgumentException("Expected " + expectedPartitions + " partitions");
 		}
-		@SuppressWarnings("unchecked")
-		Tuple2<Integer, Integer> element = next;
 
-		return element.f0;
+		return next.f0;
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/d20eda1b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
index 86b5002..4413d3f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java
@@ -26,6 +26,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import static org.junit.Assert.fail;
 
 public class TestUtils {
+	
 	public static JobExecutionResult tryExecute(StreamExecutionEnvironment see, String name) throws Exception {
 		try {
 			return see.execute(name);
@@ -45,8 +46,7 @@ public class TestUtils {
 				}
 			}
 		}
+		
 		return null;
 	}
-
-
 }


[06/14] flink git commit: [FLINK-3541] [Kafka Connector] Clean up workaround in FlinkKafkaConsumer09

Posted by se...@apache.org.
[FLINK-3541] [Kafka Connector] Clean up workaround in FlinkKafkaConsumer09

This closes #1846


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

Branch: refs/heads/master
Commit: 693d5ab09efef8732b857437bf1089f841b5e864
Parents: d20eda1
Author: Tianji Li <sk...@gmail.com>
Authored: Fri Apr 1 00:35:39 2016 -0400
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 .../connectors/kafka/FlinkKafkaConsumer09.java  | 30 ++------------------
 1 file changed, 2 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/693d5ab0/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
index d34cd2f..bc2904c 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
@@ -171,43 +171,17 @@ public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 
 		// read the partitions that belong to the listed topics
 		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-		KafkaConsumer<byte[], byte[]> consumer = null;
 
-		try {
-			consumer = new KafkaConsumer<>(this.properties);
+		try (KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(this.properties)) {
 			for (final String topic: topics) {
 				// get partitions for each topic
-				List<PartitionInfo> partitionsForTopic = null;
-				for(int tri = 0; tri < 10; tri++) {
-					LOG.info("Trying to get partitions for topic {}", topic);
-					try {
-						partitionsForTopic = consumer.partitionsFor(topic);
-						if(partitionsForTopic != null && partitionsForTopic.size() > 0) {
-							break; // it worked
-						}
-					} catch (NullPointerException npe) {
-						// workaround for KAFKA-2880: Fetcher.getTopicMetadata NullPointerException when broker cannot be reached
-						// we ignore the NPE.
-					}
-					// create a new consumer
-					consumer.close();
-					try {
-						Thread.sleep(1000);
-					} catch (InterruptedException ignored) {}
-					
-					consumer = new KafkaConsumer<>(properties);
-				}
+				List<PartitionInfo> partitionsForTopic = consumer.partitionsFor(topic);
 				// for non existing topics, the list might be null.
 				if (partitionsForTopic != null) {
 					partitions.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic));
 				}
 			}
 		}
-		finally {
-			if(consumer != null) {
-				consumer.close();
-			}
-		}
 
 		if (partitions.isEmpty()) {
 			throw new RuntimeException("Unable to retrieve any partitions for the requested topics " + topics);


[13/14] flink git commit: [FLINK-3644] [web monitor] Add new config option to set web monitor tmp dir

Posted by se...@apache.org.
[FLINK-3644] [web monitor] Add new config option to set web monitor tmp dir

This closes #1824


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

Branch: refs/heads/master
Commit: b188637b9016116728f0d16f94e213584b2abfd6
Parents: 6bb085e
Author: xueyan.li <xu...@qunar.com>
Authored: Tue Apr 5 14:10:48 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:55 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/configuration/ConfigConstants.java  |  5 +++++
 .../flink/runtime/webmonitor/HttpRequestHandler.java | 15 ++++++++-------
 .../flink/runtime/webmonitor/WebRuntimeMonitor.java  | 10 +++++-----
 .../runtime/webmonitor/WebRuntimeMonitorITCase.java  |  4 ++--
 4 files changed, 20 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b188637b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index ba2d880..53d9a37 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -435,6 +435,11 @@ public final class ConfigConstants {
 	public static final String JOB_MANAGER_WEB_PORT_KEY = "jobmanager.web.port";
 
 	/**
+	 * The config parameter defining the flink web directory to be used by the webmonitor.
+	 */
+	public static final String JOB_MANAGER_WEB_TMPDIR_KEY = "jobmanager.web.tmpdir";
+	
+	/**
 	 * The config parameter defining the number of archived jobs for the jobmanager
 	 */
 	public static final String JOB_MANAGER_WEB_ARCHIVE_COUNT = "jobmanager.web.history";

http://git-wip-us.apache.org/repos/asf/flink/blob/b188637b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
index c9190c9..bbd29fa 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/HttpRequestHandler.java
@@ -69,17 +69,18 @@ public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject>
 	
 	/** A decoder factory that always stores POST chunks on disk */
 	private static final HttpDataFactory DATA_FACTORY = new DefaultHttpDataFactory(true);
-	
-	private static final File TMP_DIR = new File(System.getProperty("java.io.tmpdir"));
-	
-		
+
+	private final File tmpDir;
+
 	private HttpRequest currentRequest;
 
 	private HttpPostRequestDecoder currentDecoder;
-	
 	private String currentRequestPath;
 
-
+	public HttpRequestHandler(File tmpDir) {
+		this.tmpDir = tmpDir;
+	}
+	
 	@Override
 	public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
 		if (currentDecoder != null) {
@@ -130,7 +131,7 @@ public class HttpRequestHandler extends SimpleChannelInboundHandler<HttpObject>
 							if (file.isCompleted()) {
 								String name = file.getFilename();
 								
-								File target = new File(TMP_DIR, UUID.randomUUID() + "_" + name);
+								File target = new File(tmpDir, UUID.randomUUID() + "_" + name);
 								file.renameTo(target);
 								
 								QueryStringEncoder encoder = new QueryStringEncoder(currentRequestPath);

http://git-wip-us.apache.org/repos/asf/flink/blob/b188637b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
index b6c578b..582004a 100644
--- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
+++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java
@@ -146,14 +146,14 @@ public class WebRuntimeMonitor implements WebMonitor {
 		
 		// create an empty directory in temp for the web server
 		String rootDirFileName = "flink-web-" + UUID.randomUUID();
-		webRootDir = new File(getBaseDir(), rootDirFileName);
+		webRootDir = new File(getBaseDir(config), rootDirFileName);
 		LOG.info("Using directory {} for the web interface files", webRootDir);
 
 		final boolean webSubmitAllow = cfg.isProgramSubmitEnabled();
 		if (webSubmitAllow) {
 			// create storage for uploads
 			String uploadDirName = "flink-web-upload-" + UUID.randomUUID();
-			this.uploadDir = new File(getBaseDir(), uploadDirName);
+			this.uploadDir = new File(getBaseDir(config), uploadDirName);
 			if (!uploadDir.mkdir() || !uploadDir.canWrite()) {
 				throw new IOException("Unable to create temporary directory to support jar uploads.");
 			}
@@ -306,7 +306,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 
 				ch.pipeline()
 						.addLast(new HttpServerCodec())
-						.addLast(new HttpRequestHandler())
+						.addLast(new HttpRequestHandler(uploadDir))
 						.addLast(handler.name(), handler)
 						.addLast(new PipelineErrorHandler(LOG));
 			}
@@ -425,7 +425,7 @@ public class WebRuntimeMonitor implements WebMonitor {
 		return new RuntimeMonitorHandler(handler, retriever, jobManagerAddressPromise.future(), timeout);
 	}
 
-	File getBaseDir() {
-		return new File(System.getProperty("java.io.tmpdir"));
+	File getBaseDir(Configuration configuration) {
+		return new File(configuration.getString(ConfigConstants.JOB_MANAGER_WEB_TMPDIR_KEY, System.getProperty("java.io.tmpdir")));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/b188637b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 8b46cdb..677ff54 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -381,7 +381,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 
 				// 2) Request file outside of web root
 				// Create a test file in the web base dir (parent of web root)
-				File illegalFile = new File(webMonitor.getBaseDir(), "test-file-" + UUID.randomUUID());
+				File illegalFile = new File(webMonitor.getBaseDir(new Configuration()), "test-file-" + UUID.randomUUID());
 				illegalFile.deleteOnExit();
 
 				assertTrue("Failed to create test file", illegalFile.createNewFile());
@@ -467,7 +467,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
 						response.getStatus());
 
 				assertFalse("Did not respond with the file, but still copied it from the JAR.",
-						new File(webMonitor.getBaseDir(), "log4j-test.properties").exists());
+						new File(webMonitor.getBaseDir(new Configuration()), "log4j-test.properties").exists());
 
 				// 3) Request non-existing file
 				client.sendGetRequest("not-existing-resource", deadline.timeLeft());


[10/14] flink git commit: [FLINK-3375] [kafka connector] Rework/simplify Kafka Connector and have a WatermarkExtractor object per partition

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

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java
new file mode 100644
index 0000000..6aaeca9
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/PeriodicOffsetCommitter.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import java.util.HashMap;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A thread that periodically writes the current Kafka partition offsets to Zookeeper.
+ */
+public class PeriodicOffsetCommitter extends Thread {
+
+	/** The ZooKeeper handler */
+	private final ZookeeperOffsetHandler offsetHandler;
+	
+	private final KafkaTopicPartitionState<?>[] partitionStates;
+	
+	/** The proxy to forward exceptions to the main thread */
+	private final ExceptionProxy errorHandler;
+	
+	/** Interval in which to commit, in milliseconds */
+	private final long commitInterval;
+	
+	/** Flag to mark the periodic committer as running */
+	private volatile boolean running = true;
+
+	PeriodicOffsetCommitter(ZookeeperOffsetHandler offsetHandler,
+			KafkaTopicPartitionState<?>[] partitionStates,
+			ExceptionProxy errorHandler,
+			long commitInterval)
+	{
+		this.offsetHandler = checkNotNull(offsetHandler);
+		this.partitionStates = checkNotNull(partitionStates);
+		this.errorHandler = checkNotNull(errorHandler);
+		this.commitInterval = commitInterval;
+		
+		checkArgument(commitInterval > 0);
+	}
+
+	@Override
+	public void run() {
+		try {
+			while (running) {
+				Thread.sleep(commitInterval);
+
+				// create copy a deep copy of the current offsets
+				HashMap<KafkaTopicPartition, Long> currentOffsets = new HashMap<>(partitionStates.length);
+				for (KafkaTopicPartitionState<?> partitionState : partitionStates) {
+					currentOffsets.put(partitionState.getKafkaTopicPartition(), partitionState.getOffset());
+				}
+				
+				offsetHandler.writeOffsets(currentOffsets);
+			}
+		}
+		catch (Throwable t) {
+			if (running) {
+				errorHandler.reportError(
+						new Exception("The periodic offset committer encountered an error: " + t.getMessage(), t));
+			}
+		}
+	}
+
+	public void shutdown() {
+		this.running = false;
+		this.interrupt();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
new file mode 100644
index 0000000..491ffad
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/SimpleConsumerThread.java
@@ -0,0 +1,504 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.internals;
+
+import kafka.api.FetchRequestBuilder;
+import kafka.api.PartitionOffsetRequestInfo;
+import kafka.common.ErrorMapping;
+import kafka.common.TopicAndPartition;
+import kafka.javaapi.FetchResponse;
+import kafka.javaapi.OffsetResponse;
+import kafka.javaapi.consumer.SimpleConsumer;
+import kafka.javaapi.message.ByteBufferMessageSet;
+import kafka.message.MessageAndOffset;
+
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.StringUtils;
+
+import org.apache.kafka.common.Node;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.flink.streaming.connectors.kafka.util.KafkaUtils.getIntFromConfig;
+
+/**
+ * This class implements a thread with a connection to a single Kafka broker. The thread
+ * pulls records for a set of topic partitions for which the connected broker is currently
+ * the leader. The thread deserializes these records and emits them. 
+ * 
+ * @param <T> The type of elements that this consumer thread creates from Kafka's byte messages
+ *            and emits into the Flink DataStream.
+ */
+class SimpleConsumerThread<T> extends Thread {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SimpleConsumerThread.class);
+
+	private static final KafkaTopicPartitionState<TopicAndPartition> MARKER = Kafka08Fetcher.MARKER;
+	
+	// ------------------------------------------------------------------------
+
+	private final Kafka08Fetcher<T> owner;
+	
+	private final KeyedDeserializationSchema<T> deserializer;
+
+	private final List<KafkaTopicPartitionState<TopicAndPartition>> partitions;
+
+	private final Node broker;
+
+	/** Queue containing new fetch partitions for the consumer thread */
+	private final ClosableBlockingQueue<KafkaTopicPartitionState<TopicAndPartition>> newPartitionsQueue;
+	
+	private final ClosableBlockingQueue<KafkaTopicPartitionState<TopicAndPartition>> unassignedPartitions;
+	
+	private final ExceptionProxy errorHandler;
+	
+	private final long invalidOffsetBehavior;
+	
+	private volatile boolean running = true;
+	
+
+	// ----------------- Simple Consumer ----------------------
+	private volatile SimpleConsumer consumer;
+
+	private final int soTimeout;
+	private final int minBytes;
+	private final int maxWait;
+	private final int fetchSize;
+	private final int bufferSize;
+	private final int reconnectLimit;
+
+
+	// exceptions are thrown locally
+	public SimpleConsumerThread(
+			Kafka08Fetcher<T> owner,
+			ExceptionProxy errorHandler,
+			Properties config,
+			Node broker,
+			List<KafkaTopicPartitionState<TopicAndPartition>> seedPartitions,
+			ClosableBlockingQueue<KafkaTopicPartitionState<TopicAndPartition>> unassignedPartitions,
+			KeyedDeserializationSchema<T> deserializer,
+			long invalidOffsetBehavior)
+	{
+		this.owner = owner;
+		this.errorHandler = errorHandler;
+		this.broker = broker;
+		this.partitions = seedPartitions;
+		this.deserializer = requireNonNull(deserializer);
+		this.unassignedPartitions = requireNonNull(unassignedPartitions);
+		this.newPartitionsQueue = new ClosableBlockingQueue<>();
+		this.invalidOffsetBehavior = invalidOffsetBehavior;
+		
+		// these are the actual configuration values of Kafka + their original default values.
+		this.soTimeout = getIntFromConfig(config, "socket.timeout.ms", 30000);
+		this.minBytes = getIntFromConfig(config, "fetch.min.bytes", 1);
+		this.maxWait = getIntFromConfig(config, "fetch.wait.max.ms", 100);
+		this.fetchSize = getIntFromConfig(config, "fetch.message.max.bytes", 1048576);
+		this.bufferSize = getIntFromConfig(config, "socket.receive.buffer.bytes", 65536);
+		this.reconnectLimit = getIntFromConfig(config, "flink.simple-consumer-reconnectLimit", 3);
+	}
+
+	public ClosableBlockingQueue<KafkaTopicPartitionState<TopicAndPartition>> getNewPartitionsQueue() {
+		return newPartitionsQueue;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  main work loop
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public void run() {
+		LOG.info("Starting to fetch from {}", this.partitions);
+
+		// set up the config values
+		final String clientId = "flink-kafka-consumer-legacy-" + broker.id();
+
+		try {
+			// create the Kafka consumer that we actually use for fetching
+			consumer = new SimpleConsumer(broker.host(), broker.port(), soTimeout, bufferSize, clientId);
+			
+			// make sure that all partitions have some offsets to start with
+			// those partitions that do not have an offset from a checkpoint need to get
+			// their start offset from ZooKeeper
+			getMissingOffsetsFromKafka(partitions);
+
+			// Now, the actual work starts :-)
+			int offsetOutOfRangeCount = 0;
+			int reconnects = 0;
+			while (running) {
+
+				// ----------------------------------- partitions list maintenance ----------------------------
+
+				// check queue for new partitions to read from:
+				List<KafkaTopicPartitionState<TopicAndPartition>> newPartitions = newPartitionsQueue.pollBatch();
+				if (newPartitions != null) {
+					// found some new partitions for this thread's broker
+					
+					// check if the new partitions need an offset lookup
+					getMissingOffsetsFromKafka(newPartitions);
+					
+					// add the new partitions (and check they are not already in there)
+					for (KafkaTopicPartitionState<TopicAndPartition> newPartition: newPartitions) {
+						if (partitions.contains(newPartition)) {
+							throw new IllegalStateException("Adding partition " + newPartition + 
+									" to subscribed partitions even though it is already subscribed");
+						}
+						partitions.add(newPartition);
+					}
+					
+					LOG.info("Adding {} new partitions to consumer thread {}", newPartitions.size(), getName());
+					LOG.debug("Partitions list: {}", newPartitions);
+				}
+
+				if (partitions.size() == 0) {
+					if (newPartitionsQueue.close()) {
+						// close succeeded. Closing thread
+						running = false;
+						
+						LOG.info("Consumer thread {} does not have any partitions assigned anymore. Stopping thread.", 
+								getName());
+
+						// add the wake-up marker into the queue to make the main thread
+						// immediately wake up and termination faster
+						unassignedPartitions.add(MARKER);
+
+						break;
+					} else {
+						// close failed: fetcher main thread concurrently added new partitions into the queue.
+						// go to top of loop again and get the new partitions
+						continue; 
+					}
+				}
+
+				// ----------------------------------- request / response with kafka ----------------------------
+
+				FetchRequestBuilder frb = new FetchRequestBuilder();
+				frb.clientId(clientId);
+				frb.maxWait(maxWait);
+				frb.minBytes(minBytes);
+
+				for (KafkaTopicPartitionState<?> partition : partitions) {
+					frb.addFetch(
+							partition.getKafkaTopicPartition().getTopic(),
+							partition.getKafkaTopicPartition().getPartition(),
+							partition.getOffset() + 1, // request the next record
+							fetchSize);
+				}
+				
+				kafka.api.FetchRequest fetchRequest = frb.build();
+				LOG.debug("Issuing fetch request {}", fetchRequest);
+
+				FetchResponse fetchResponse;
+				try {
+					fetchResponse = consumer.fetch(fetchRequest);
+				}
+				catch (Throwable cce) {
+					//noinspection ConstantConditions
+					if (cce instanceof ClosedChannelException) {
+						LOG.warn("Fetch failed because of ClosedChannelException.");
+						LOG.debug("Full exception", cce);
+						
+						// we don't know if the broker is overloaded or unavailable.
+						// retry a few times, then return ALL partitions for new leader lookup
+						if (++reconnects >= reconnectLimit) {
+							LOG.warn("Unable to reach broker after {} retries. Returning all current partitions", reconnectLimit);
+							for (KafkaTopicPartitionState<TopicAndPartition> fp: this.partitions) {
+								unassignedPartitions.add(fp);
+							}
+							this.partitions.clear();
+							continue; // jump to top of loop: will close thread or subscribe to new partitions
+						}
+						try {
+							consumer.close();
+						} catch (Throwable t) {
+							LOG.warn("Error while closing consumer connection", t);
+						}
+						// delay & retry
+						Thread.sleep(100);
+						consumer = new SimpleConsumer(broker.host(), broker.port(), soTimeout, bufferSize, clientId);
+						continue; // retry
+					} else {
+						throw cce;
+					}
+				}
+				reconnects = 0;
+
+				// ---------------------------------------- error handling ----------------------------
+
+				if (fetchResponse == null) {
+					throw new IOException("Fetch from Kafka failed (request returned null)");
+				}
+				
+				if (fetchResponse.hasError()) {
+					String exception = "";
+					List<KafkaTopicPartitionState<TopicAndPartition>> partitionsToGetOffsetsFor = new ArrayList<>();
+					
+					// iterate over partitions to get individual error codes
+					Iterator<KafkaTopicPartitionState<TopicAndPartition>> partitionsIterator = partitions.iterator();
+					boolean partitionsRemoved = false;
+					
+					while (partitionsIterator.hasNext()) {
+						final KafkaTopicPartitionState<TopicAndPartition> fp = partitionsIterator.next();
+						short code = fetchResponse.errorCode(fp.getTopic(), fp.getPartition());
+
+						if (code == ErrorMapping.OffsetOutOfRangeCode()) {
+							// we were asked to read from an out-of-range-offset (maybe set wrong in Zookeeper)
+							// Kafka's high level consumer is resetting the offset according to 'auto.offset.reset'
+							partitionsToGetOffsetsFor.add(fp);
+						}
+						else if (code == ErrorMapping.NotLeaderForPartitionCode() ||
+								code == ErrorMapping.LeaderNotAvailableCode() ||
+								code == ErrorMapping.BrokerNotAvailableCode() ||
+								code == ErrorMapping.UnknownCode())
+						{
+							// the broker we are connected to is not the leader for the partition.
+							LOG.warn("{} is not the leader of {}. Reassigning leader for partition", broker, fp);
+							LOG.debug("Error code = {}", code);
+
+							unassignedPartitions.add(fp);
+
+							partitionsIterator.remove(); // unsubscribe the partition ourselves
+							partitionsRemoved = true;
+						}
+						else if (code != ErrorMapping.NoError()) {
+							exception += "\nException for " + fp.getTopic() +":"+ fp.getPartition() + ": " +
+									StringUtils.stringifyException(ErrorMapping.exceptionFor(code));
+						}
+					}
+					if (partitionsToGetOffsetsFor.size() > 0) {
+						// safeguard against an infinite loop.
+						if (offsetOutOfRangeCount++ > 3) {
+							throw new RuntimeException("Found invalid offsets more than three times in partitions "
+									+ partitionsToGetOffsetsFor + " Exceptions: " + exception);
+						}
+						// get valid offsets for these partitions and try again.
+						LOG.warn("The following partitions had an invalid offset: {}", partitionsToGetOffsetsFor);
+						getLastOffsetFromKafka(consumer, partitionsToGetOffsetsFor, invalidOffsetBehavior);
+						
+						LOG.warn("The new partition offsets are {}", partitionsToGetOffsetsFor);
+						continue; // jump back to create a new fetch request. The offset has not been touched.
+					}
+					else if (partitionsRemoved) {
+						continue; // create new fetch request
+					}
+					else {
+						// partitions failed on an error
+						throw new IOException("Error while fetching from broker '" + broker +"': " + exception);
+					}
+				} else {
+					// successful fetch, reset offsetOutOfRangeCount.
+					offsetOutOfRangeCount = 0;
+				}
+
+				// ----------------------------------- process fetch response ----------------------------
+
+				int messagesInFetch = 0;
+				int deletedMessages = 0;
+				Iterator<KafkaTopicPartitionState<TopicAndPartition>> partitionsIterator = partitions.iterator();
+				
+				partitionsLoop:
+				while (partitionsIterator.hasNext()) {
+					final KafkaTopicPartitionState<TopicAndPartition> currentPartition = partitionsIterator.next();
+					
+					final ByteBufferMessageSet messageSet = fetchResponse.messageSet(
+							currentPartition.getTopic(), currentPartition.getPartition());
+
+					for (MessageAndOffset msg : messageSet) {
+						if (running) {
+							messagesInFetch++;
+							final ByteBuffer payload = msg.message().payload();
+							final long offset = msg.offset();
+							
+							if (offset <= currentPartition.getOffset()) {
+								// we have seen this message already
+								LOG.info("Skipping message with offset " + msg.offset()
+										+ " because we have seen messages until (including) "
+										+ currentPartition.getOffset()
+										+ " from topic/partition " + currentPartition.getTopic() + '/'
+										+ currentPartition.getPartition() + " already");
+								continue;
+							}
+
+							// If the message value is null, this represents a delete command for the message key.
+							// Log this and pass it on to the client who might want to also receive delete messages.
+							byte[] valueBytes;
+							if (payload == null) {
+								deletedMessages++;
+								valueBytes = null;
+							} else {
+								valueBytes = new byte[payload.remaining()];
+								payload.get(valueBytes);
+							}
+
+							// put key into byte array
+							byte[] keyBytes = null;
+							int keySize = msg.message().keySize();
+
+							if (keySize >= 0) { // message().hasKey() is doing the same. We save one int deserialization
+								ByteBuffer keyPayload = msg.message().key();
+								keyBytes = new byte[keySize];
+								keyPayload.get(keyBytes);
+							}
+
+							final T value = deserializer.deserialize(keyBytes, valueBytes, 
+									currentPartition.getTopic(), currentPartition.getPartition(), offset);
+							
+							if (deserializer.isEndOfStream(value)) {
+								// remove partition from subscribed partitions.
+								partitionsIterator.remove();
+								continue partitionsLoop;
+							}
+							
+							owner.emitRecord(value, currentPartition, offset);
+						}
+						else {
+							// no longer running
+							return;
+						}
+					}
+				}
+				LOG.debug("This fetch contained {} messages ({} deleted messages)", messagesInFetch, deletedMessages);
+			} // end of fetch loop
+
+			if (!newPartitionsQueue.close()) {
+				throw new Exception("Bug: Cleanly leaving fetcher thread without having a closed queue.");
+			}
+		}
+		catch (Throwable t) {
+			// report to the fetcher's error handler
+			errorHandler.reportError(t);
+		}
+		finally {
+			if (consumer != null) {
+				// closing the consumer should not fail the program
+				try {
+					consumer.close();
+				}
+				catch (Throwable t) {
+					LOG.error("Error while closing the Kafka simple consumer", t);
+				}
+			}
+		}
+	}
+
+	private void getMissingOffsetsFromKafka(
+			List<KafkaTopicPartitionState<TopicAndPartition>> partitions) throws IOException
+	{
+		// collect which partitions we should fetch offsets for
+		List<KafkaTopicPartitionState<TopicAndPartition>> partitionsToGetOffsetsFor = new ArrayList<>();
+		for (KafkaTopicPartitionState<TopicAndPartition> part : partitions) {
+			if (!part.isOffsetDefined()) {
+				// retrieve the offset from the consumer
+				partitionsToGetOffsetsFor.add(part);
+			}
+		}
+		
+		if (partitionsToGetOffsetsFor.size() > 0) {
+			getLastOffsetFromKafka(consumer, partitionsToGetOffsetsFor, invalidOffsetBehavior);
+			
+			LOG.info("No checkpoint/savepoint offsets found for some partitions. " +
+					"Fetched the following start offsets {}", partitionsToGetOffsetsFor);
+		}
+	}
+
+	/**
+	 * Cancels this fetch thread. The thread will release all resources and terminate.
+	 */
+	public void cancel() {
+		this.running = false;
+
+		// interrupt whatever the consumer is doing
+		if (consumer != null) {
+			consumer.close();
+		}
+
+		this.interrupt();
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Kafka Request Utils
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Request latest offsets for a set of partitions, via a Kafka consumer.
+	 *
+	 * <p>This method retries three times if the response has an error.
+	 *
+	 * @param consumer The consumer connected to lead broker
+	 * @param partitions The list of partitions we need offsets for
+	 * @param whichTime The type of time we are requesting. -1 and -2 are special constants (See OffsetRequest)
+	 */
+	private static void getLastOffsetFromKafka(
+			SimpleConsumer consumer,
+			List<KafkaTopicPartitionState<TopicAndPartition>> partitions,
+			long whichTime) throws IOException
+	{
+		Map<TopicAndPartition, PartitionOffsetRequestInfo> requestInfo = new HashMap<>();
+		for (KafkaTopicPartitionState<TopicAndPartition> part : partitions) {
+			requestInfo.put(part.getKafkaPartitionHandle(), new PartitionOffsetRequestInfo(whichTime, 1));
+		}
+
+		int retries = 0;
+		OffsetResponse response;
+		while (true) {
+			kafka.javaapi.OffsetRequest request = new kafka.javaapi.OffsetRequest(
+					requestInfo, kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId());
+			response = consumer.getOffsetsBefore(request);
+
+			if (response.hasError()) {
+				StringBuilder exception = new StringBuilder();
+				for (KafkaTopicPartitionState<TopicAndPartition> part : partitions) {
+					short code;
+					if ((code = response.errorCode(part.getTopic(), part.getPartition())) != ErrorMapping.NoError()) {
+						exception.append("\nException for topic=").append(part.getTopic())
+								.append(" partition=").append(part.getPartition()).append(": ")
+								.append(StringUtils.stringifyException(ErrorMapping.exceptionFor(code)));
+					}
+				}
+				if (++retries >= 3) {
+					throw new IOException("Unable to get last offset for partitions " + partitions + ": "
+							+ exception.toString());
+				} else {
+					LOG.warn("Unable to get last offset for partitions: Exception(s): {}", exception);
+				}
+			} else {
+				break; // leave retry loop
+			}
+		}
+
+		for (KafkaTopicPartitionState<TopicAndPartition> part: partitions) {
+			final long offset = response.offsets(part.getTopic(), part.getPartition())[0];
+			
+			// the offset returned is that of the next record to fetch. because our state reflects the latest
+			// successfully emitted record, we subtract one
+			part.setOffset(offset - 1);
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
index 328cab0..a1a81ed 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/ZookeeperOffsetHandler.java
@@ -24,7 +24,6 @@ import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.retry.ExponentialBackoffRetry;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 
 import org.slf4j.Logger;
@@ -39,11 +38,9 @@ import java.util.Properties;
 /**
  * Handler for committing Kafka offsets to Zookeeper and to retrieve them again.
  */
-public class ZookeeperOffsetHandler implements OffsetHandler {
+public class ZookeeperOffsetHandler {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ZookeeperOffsetHandler.class);
-	
-	private static final long OFFSET_NOT_SET = FlinkKafkaConsumerBase.OFFSET_NOT_SET;
 
 	private final String groupId;
 
@@ -74,27 +71,40 @@ public class ZookeeperOffsetHandler implements OffsetHandler {
 		curatorClient = CuratorFrameworkFactory.newClient(zkConnect, sessionTimeoutMs, connectionTimeoutMs, retryPolicy);
 		curatorClient.start();
 	}
+	
+	// ------------------------------------------------------------------------
+	//  Offset access and manipulation
+	// ------------------------------------------------------------------------
 
-
-	@Override
-	public void commit(Map<KafkaTopicPartition, Long> offsetsToCommit) throws Exception {
-		for (Map.Entry<KafkaTopicPartition, Long> entry : offsetsToCommit.entrySet()) {
+	/**
+	 * Writes given set of offsets for Kafka partitions to ZooKeeper.
+	 * 
+	 * @param offsetsToWrite The offsets for the partitions to write.
+	 * @throws Exception The method forwards exceptions.
+	 */
+	public void writeOffsets(Map<KafkaTopicPartition, Long> offsetsToWrite) throws Exception {
+		for (Map.Entry<KafkaTopicPartition, Long> entry : offsetsToWrite.entrySet()) {
 			KafkaTopicPartition tp = entry.getKey();
 			long offset = entry.getValue();
-			
+
 			if (offset >= 0) {
 				setOffsetInZooKeeper(curatorClient, groupId, tp.getTopic(), tp.getPartition(), offset);
 			}
 		}
 	}
 
-	@Override
+	/**
+	 * 
+	 * @param partitions The partitions to read offsets for.
+	 * @return The mapping from partition to offset.
+	 * @throws Exception This method forwards exceptions.
+	 */
 	public Map<KafkaTopicPartition, Long> getOffsets(List<KafkaTopicPartition> partitions) throws Exception {
 		Map<KafkaTopicPartition, Long> ret = new HashMap<>(partitions.size());
 		for (KafkaTopicPartition tp : partitions) {
-			long offset = getOffsetFromZooKeeper(curatorClient, groupId, tp.getTopic(), tp.getPartition());
+			Long offset = getOffsetFromZooKeeper(curatorClient, groupId, tp.getTopic(), tp.getPartition());
 
-			if (offset != OFFSET_NOT_SET) {
+			if (offset != null) {
 				LOG.info("Offset for TopicPartition {}:{} was set to {} in ZooKeeper. Seeking fetcher to that position.",
 						tp.getTopic(), tp.getPartition(), offset);
 				ret.put(tp, offset);
@@ -103,7 +113,11 @@ public class ZookeeperOffsetHandler implements OffsetHandler {
 		return ret;
 	}
 
-	@Override
+	/**
+	 * Closes the offset handler.
+	 * 
+	 * @throws IOException Thrown, if the handler cannot be closed properly.
+	 */
 	public void close() throws IOException {
 		curatorClient.close();
 	}
@@ -120,7 +134,7 @@ public class ZookeeperOffsetHandler implements OffsetHandler {
 		curatorClient.setData().forPath(path, data);
 	}
 
-	public static long getOffsetFromZooKeeper(CuratorFramework curatorClient, String groupId, String topic, int partition) throws Exception {
+	public static Long getOffsetFromZooKeeper(CuratorFramework curatorClient, String groupId, String topic, int partition) throws Exception {
 		ZKGroupTopicDirs topicDirs = new ZKGroupTopicDirs(groupId, topic);
 		String path = topicDirs.consumerOffsetDir() + "/" + partition;
 		curatorClient.newNamespaceAwareEnsurePath(path).ensure(curatorClient.getZookeeperClient());
@@ -128,18 +142,20 @@ public class ZookeeperOffsetHandler implements OffsetHandler {
 		byte[] data = curatorClient.getData().forPath(path);
 		
 		if (data == null) {
-			return OFFSET_NOT_SET;
+			return null;
 		} else {
 			String asString = new String(data);
 			if (asString.length() == 0) {
-				return OFFSET_NOT_SET;
+				return null;
 			} else {
 				try {
-					return Long.parseLong(asString);
-				} catch (NumberFormatException e) {
-					throw new Exception(String.format(
-						"The offset in ZooKeeper for group '%s', topic '%s', partition %d is a malformed string: %s",
-						groupId, topic, partition, asString));
+					return Long.valueOf(asString);
+				}
+				catch (NumberFormatException e) {
+					LOG.error(
+							"The offset in ZooKeeper for group '{}', topic '{}', partition {} is a malformed string: {}",
+						groupId, topic, partition, asString);
+					return null;
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
index d6ee968..0aef3bd 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java
@@ -22,7 +22,6 @@ import org.apache.curator.framework.recipes.cache.TreeCache;
 import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
 import org.apache.curator.framework.recipes.cache.TreeCacheListener;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
-import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.runtime.client.JobCancellationException;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -35,8 +34,11 @@ import org.junit.Test;
 
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class Kafka08ITCase extends KafkaConsumerTestBase {
 
@@ -45,11 +47,6 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 	// ------------------------------------------------------------------------
 
 	@Test(timeout = 60000)
-	public void testCheckpointing() throws Exception {
-		runCheckpointingTest();
-	}
-
-	@Test(timeout = 60000)
 	public void testFailOnNoBroker() throws Exception {
 		runFailOnNoBrokerTest();
 	}
@@ -60,15 +57,15 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		runSimpleConcurrentProducerConsumerTopology();
 	}
 
-	@Test(timeout = 60000)
-	public void testPunctuatedExplicitWMConsumer() throws Exception {
-		runExplicitPunctuatedWMgeneratingConsumerTest(false);
-	}
+//	@Test(timeout = 60000)
+//	public void testPunctuatedExplicitWMConsumer() throws Exception {
+//		runExplicitPunctuatedWMgeneratingConsumerTest(false);
+//	}
 
-	@Test(timeout = 60000)
-	public void testPunctuatedExplicitWMConsumerWithEmptyTopic() throws Exception {
-		runExplicitPunctuatedWMgeneratingConsumerTest(true);
-	}
+//	@Test(timeout = 60000)
+//	public void testPunctuatedExplicitWMConsumerWithEmptyTopic() throws Exception {
+//		runExplicitPunctuatedWMgeneratingConsumerTest(true);
+//	}
 
 	@Test(timeout = 60000)
 	public void testKeyValueSupport() throws Exception {
@@ -164,7 +161,31 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		runMetricsAndEndOfStreamTest();
 	}
 
+	@Test
+	public void runOffsetManipulationInZooKeeperTest() {
+		try {
+			final String topicName = "ZookeeperOffsetHandlerTest-Topic";
+			final String groupId = "ZookeeperOffsetHandlerTest-Group";
 
+			final Long offset = (long) (Math.random() * Long.MAX_VALUE);
+
+			CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer ).createCuratorClient();
+			kafkaServer.createTestTopic(topicName, 3, 2);
+
+			ZookeeperOffsetHandler.setOffsetInZooKeeper(curatorFramework, groupId, topicName, 0, offset);
+
+			Long fetchedOffset = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, groupId, topicName, 0);
+
+			curatorFramework.close();
+
+			assertEquals(offset, fetchedOffset);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
 	/**
 	 * Tests that offsets are properly committed to ZooKeeper and initial offsets are read from ZooKeeper.
 	 *
@@ -202,15 +223,15 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 
 		CuratorFramework curatorClient = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
 
-		long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardProps.getProperty("group.id"), topicName, 0);
-		long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardProps.getProperty("group.id"), topicName, 1);
-		long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardProps.getProperty("group.id"), topicName, 2);
+		Long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardProps.getProperty("group.id"), topicName, 0);
+		Long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardProps.getProperty("group.id"), topicName, 1);
+		Long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorClient, standardProps.getProperty("group.id"), topicName, 2);
 
 		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
 
-		assertTrue(o1 == FlinkKafkaConsumerBase.OFFSET_NOT_SET || (o1 >= 0 && o1 <= 100));
-		assertTrue(o2 == FlinkKafkaConsumerBase.OFFSET_NOT_SET || (o2 >= 0 && o2 <= 100));
-		assertTrue(o3 == FlinkKafkaConsumerBase.OFFSET_NOT_SET || (o3 >= 0 && o3 <= 100));
+		assertTrue(o1 == null || (o1 >= 0 && o1 <= 100));
+		assertTrue(o2 == null || (o2 >= 0 && o2 <= 100));
+		assertTrue(o3 == null || (o3 >= 0 && o3 <= 100));
 
 		LOG.info("Manipulating offsets");
 
@@ -264,16 +285,16 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		// get the offset
 		CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
 
-		long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 0);
-		long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 1);
-		long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 2);
+		Long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 0);
+		Long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 1);
+		Long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 2);
 		curatorFramework.close();
 		LOG.info("Got final offsets from zookeeper o1={}, o2={}, o3={}", o1, o2, o3);
 
 		// ensure that the offset has been committed
-		boolean atLeastOneOffsetSet = (o1 > 0 && o1 <= 100) ||
-				(o2 > 0 && o2 <= 100) ||
-				(o3 > 0 && o3 <= 100);
+		boolean atLeastOneOffsetSet = (o1 != null && o1 > 0 && o1 <= 100) ||
+				(o2 != null && o2 > 0 && o2 <= 100) ||
+				(o3 != null && o3 > 0 && o3 <= 100);
 		assertTrue("Expecting at least one offset to be set o1="+o1+" o2="+o2+" o3="+o3, atLeastOneOffsetSet);
 
 		deleteTestTopic(topicName);
@@ -295,7 +316,9 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 	public void testKafkaOffsetRetrievalToZookeeper() throws Exception {
 		final String topicName = "testKafkaOffsetToZk";
 		final int parallelism = 3;
+		
 		createTestTopic(topicName, parallelism, 1);
+		
 		StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
 		env1.getConfig().disableSysoutLogging();
 		env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
@@ -305,9 +328,7 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		writeSequence(env1, topicName, 50, parallelism);
 
 
-		StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
-
-		// enable checkpointing
+		final StreamExecutionEnvironment env2 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
 		env2.getConfig().disableSysoutLogging();
 		env2.getConfig().setRestartStrategy(RestartStrategies.noRestart());
 		env2.setParallelism(parallelism);
@@ -320,78 +341,55 @@ public class Kafka08ITCase extends KafkaConsumerTestBase {
 		DataStream<String> stream = env2.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), readProps));
 		stream.addSink(new DiscardingSink<String>());
 
-		CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
-		String consumerGroupDir = standardProps.getProperty("group.id");
-		TreeCache tc1 = new TreeCache(curatorFramework, "/consumers/" + consumerGroupDir + "/offsets/" + topicName + "/0");
-		TreeCache tc2 = new TreeCache(curatorFramework, "/consumers/" + consumerGroupDir + "/offsets/" + topicName + "/1");
-		TreeCache tc3 = new TreeCache(curatorFramework, "/consumers/" + consumerGroupDir + "/offsets/" + topicName + "/2");
-
-		// add listener to wait until first partition is updated in ZK
-		TreeCacheListener stopListener = new TreeCacheListener() {
-			AtomicInteger counter = new AtomicInteger(0);
-			@Override
-			public void childEvent(CuratorFramework client, TreeCacheEvent event) throws Exception {
-				LOG.info("Updated {}", event);
-				if (event.getType().equals(TreeCacheEvent.Type.NODE_UPDATED)) {
-					if(counter.incrementAndGet() == 3) {
-						// cancel job, node has been created
-						LOG.info("Cancelling job after all three ZK nodes were updated");
-						JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
-					}
-				}
-			}
-		};
-		tc1.getListenable().addListener(stopListener);
-		tc1.start();
-		tc2.getListenable().addListener(stopListener);
-		tc2.start();
-		tc3.getListenable().addListener(stopListener);
-		tc3.start();
-
-		// the curator listener is not always working properly. Stop job after 10 seconds
-		final Tuple1<Throwable> error = new Tuple1<>();
-		Thread canceller = new Thread(new Runnable() {
+		final AtomicReference<Throwable> errorRef = new AtomicReference<>();
+		final Thread runner = new Thread("runner") {
 			@Override
 			public void run() {
 				try {
-					Thread.sleep(10_000L);
-					LOG.info("Cancelling job after 10 seconds");
-					JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
-				} catch (Throwable t) {
-					if (!(t instanceof InterruptedException)) {
-						error.f0 = t;
+					env2.execute();
+				}
+				catch (Throwable t) {
+					if (!(t.getCause() instanceof JobCancellationException)) {
+						errorRef.set(t);
 					}
 				}
 			}
-		});
-		canceller.start();
-
-		try {
-			env2.execute("Idlying Kafka source");
-		} catch( Throwable thr) {
-			if(!(thr.getCause() instanceof JobCancellationException)) {
-				throw thr;
+		};
+		runner.start();
+
+		final CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl)kafkaServer).createCuratorClient();
+		final Long l49 = 49L;
+				
+		final long deadline = 30000 + System.currentTimeMillis();
+		do {
+			Long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 0);
+			Long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 1);
+			Long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 2);
+
+			if (l49.equals(o1) && l49.equals(o2) && l49.equals(o3)) {
+				break;
 			}
+			
+			Thread.sleep(100);
 		}
-		tc1.close();
-		tc2.close();
-		tc3.close();
-
-		canceller.interrupt();
-		canceller.join();
-		if(error.f0 != null) {
-			throw new RuntimeException("Delayed cancelling thread had an error", error.f0);
+		while (System.currentTimeMillis() < deadline);
+		
+		// cancel the job
+		JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
+		
+		final Throwable t = errorRef.get();
+		if (t != null) {
+			throw new RuntimeException("Job failed with an exception", t);
 		}
 
 		// check if offsets are correctly in ZK
-		long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 0);
-		long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 1);
-		long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 2);
-		Assert.assertEquals(49L, o1);
-		Assert.assertEquals(49L, o2);
-		Assert.assertEquals(49L, o3);
+		Long o1 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 0);
+		Long o2 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 1);
+		Long o3 = ZookeeperOffsetHandler.getOffsetFromZooKeeper(curatorFramework, standardProps.getProperty("group.id"), topicName, 2);
+		Assert.assertEquals(Long.valueOf(49L), o1);
+		Assert.assertEquals(Long.valueOf(49L), o2);
+		Assert.assertEquals(Long.valueOf(49L), o3);
 
 		curatorFramework.close();
-
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
new file mode 100644
index 0000000..36fb7e6
--- /dev/null
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka;
+
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.junit.Assert.*;
+
+public class KafkaConsumer08Test {
+
+	@Test
+	public void testValidateZooKeeperConfig() {
+		try {
+			// empty
+			Properties emptyProperties = new Properties();
+			try {
+				FlinkKafkaConsumer08.validateZooKeeperConfig(emptyProperties);
+				fail("should fail with an exception");
+			}
+			catch (IllegalArgumentException e) {
+				// expected
+			}
+
+			// no connect string (only group string)
+			Properties noConnect = new Properties();
+			noConnect.put(ConsumerConfig.GROUP_ID_CONFIG, "flink-test-group");
+			try {
+				FlinkKafkaConsumer08.validateZooKeeperConfig(noConnect);
+				fail("should fail with an exception");
+			}
+			catch (IllegalArgumentException e) {
+				// expected
+			}
+
+			// no group string (only connect string)
+			Properties noGroup = new Properties();
+			noGroup.put("zookeeper.connect", "localhost:47574");
+			try {
+				FlinkKafkaConsumer08.validateZooKeeperConfig(noGroup);
+				fail("should fail with an exception");
+			}
+			catch (IllegalArgumentException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCreateSourceWithoutCluster() {
+		try {
+			Properties props = new Properties();
+			props.setProperty("zookeeper.connect", "localhost:56794");
+			props.setProperty("bootstrap.servers", "localhost:11111, localhost:22222");
+			props.setProperty("group.id", "non-existent-group");
+
+			new FlinkKafkaConsumer08<>(Collections.singletonList("no op topic"), new SimpleStringSchema(), props);
+			fail();
+		}
+		catch (Exception e) {
+			assertTrue(e.getMessage().contains("Unable to retrieve any partitions"));
+		}
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
index 21140dd..c28799c 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetention08ITCase.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.junit.Test;
 
+@SuppressWarnings("serial")
 public class KafkaShortRetention08ITCase extends KafkaShortRetentionTestBase {
 
 	@Test(timeout=60000)
@@ -28,6 +29,6 @@ public class KafkaShortRetention08ITCase extends KafkaShortRetentionTestBase {
 
 	@Test(timeout=60000)
 	public void testAutoOffsetResetNone() throws Exception {
-		runFailOnAutoOffsetResetNone();
+		runFailOnAutoOffsetResetNoneEager();
 	}
 }

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

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties
index 6bdfb48..fbeb110 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties
+++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/resources/log4j-test.properties
@@ -25,5 +25,6 @@ log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
 
 # suppress the irrelevant (wrong) warnings from the netty channel handler
 log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
-
-
+log4j.logger.org.apache.zookeeper=OFF, testlogger
+log4j.logger.state.change.logger=OFF, testlogger
+log4j.logger.kafka=OFF, testlogger

http://git-wip-us.apache.org/repos/asf/flink/blob/3c93103d/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
index 55f9875..d34cd2f 100644
--- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
+++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
@@ -17,38 +17,31 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.metrics.DefaultKafkaMetricAccumulator;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
+import org.apache.flink.util.SerializedValue;
+
 import org.apache.kafka.clients.consumer.ConsumerConfig;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.PartitionInfo;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.WakeupException;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
-import java.util.UUID;
 
-import static java.util.Objects.requireNonNull;
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
@@ -73,10 +66,8 @@ import static java.util.Objects.requireNonNull;
  */
 public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 
-	// ------------------------------------------------------------------------
-	
 	private static final long serialVersionUID = 2324564345203409112L;
-	
+
 	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaConsumer09.class);
 
 	/**  Configuration key to change the polling timeout **/
@@ -85,35 +76,18 @@ public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 	/** Boolean configuration key to disable metrics tracking **/
 	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
 
-	/**
-	 * From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
-	 * available. If 0, returns immediately with any records that are available now.
-	 */
+	/** From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
+	 * available. If 0, returns immediately with any records that are available now. */
 	public static final long DEFAULT_POLL_TIMEOUT = 100L;
 
+	// ------------------------------------------------------------------------
+
 	/** User-supplied properties for Kafka **/
 	private final Properties properties;
-	/** Ordered list of all partitions available in all subscribed partitions **/
-	private final List<KafkaTopicPartition> partitionInfos;
-
-	/** Unique ID identifying the consumer */
-	private final String consumerId;
-
-	// ------  Runtime State  -------
-
-	/** The partitions actually handled by this consumer at runtime */
-	private transient List<TopicPartition> subscribedPartitions;
-	/** For performance reasons, we are keeping two representations of the subscribed partitions **/
-	private transient List<KafkaTopicPartition> subscribedPartitionsAsFlink;
-	/** The Kafka Consumer instance**/
-	private transient KafkaConsumer<byte[], byte[]> consumer;
-	/** The thread running Kafka's consumer **/
-	private transient ConsumerThread<T> consumerThread;
-	/** Exception set from the ConsumerThread */
-	private transient Throwable consumerThreadException;
-	/** If the consumer doesn't have a Kafka partition assigned at runtime, it'll block on this waitThread **/
-	private transient Thread waitThread;
 
+	/** From Kafka's Javadoc: The time, in milliseconds, spent waiting in poll if data is not
+	 * available. If 0, returns immediately with any records that are available now */
+	private final long pollTimeout;
 
 	// ------------------------------------------------------------------------
 
@@ -177,14 +151,30 @@ public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 	 *           The properties that are used to configure both the fetcher and the offset handler.
 	 */
 	public FlinkKafkaConsumer09(List<String> topics, KeyedDeserializationSchema<T> deserializer, Properties props) {
-		super(deserializer, props);
-		requireNonNull(topics, "topics");
-		this.properties = requireNonNull(props, "props");
+		super(deserializer);
+
+		checkNotNull(topics, "topics");
+		this.properties = checkNotNull(props, "props");
 		setDeserializer(this.properties);
+
+		// configure the polling timeout
+		try {
+			if (properties.containsKey(KEY_POLL_TIMEOUT)) {
+				this.pollTimeout = Long.parseLong(properties.getProperty(KEY_POLL_TIMEOUT));
+			} else {
+				this.pollTimeout = DEFAULT_POLL_TIMEOUT;
+			}
+		}
+		catch (Exception e) {
+			throw new IllegalArgumentException("Cannot parse poll timeout for '" + KEY_POLL_TIMEOUT + '\'', e);
+		}
+
+		// read the partitions that belong to the listed topics
+		final List<KafkaTopicPartition> partitions = new ArrayList<>();
 		KafkaConsumer<byte[], byte[]> consumer = null;
+
 		try {
 			consumer = new KafkaConsumer<>(this.properties);
-			this.partitionInfos = new ArrayList<>();
 			for (final String topic: topics) {
 				// get partitions for each topic
 				List<PartitionInfo> partitionsForTopic = null;
@@ -203,307 +193,93 @@ public class FlinkKafkaConsumer09<T> extends FlinkKafkaConsumerBase<T> {
 					consumer.close();
 					try {
 						Thread.sleep(1000);
-					} catch (InterruptedException e) {
-					}
+					} catch (InterruptedException ignored) {}
+					
 					consumer = new KafkaConsumer<>(properties);
 				}
 				// for non existing topics, the list might be null.
-				if(partitionsForTopic != null) {
-					partitionInfos.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic));
+				if (partitionsForTopic != null) {
+					partitions.addAll(convertToFlinkKafkaTopicPartition(partitionsForTopic));
 				}
 			}
-		} finally {
+		}
+		finally {
 			if(consumer != null) {
 				consumer.close();
 			}
 		}
-		if(partitionInfos.isEmpty()) {
+
+		if (partitions.isEmpty()) {
 			throw new RuntimeException("Unable to retrieve any partitions for the requested topics " + topics);
 		}
 
 		// we now have a list of partitions which is the same for all parallel consumer instances.
-		LOG.info("Got {} partitions from these topics: {}", partitionInfos.size(), topics);
+		LOG.info("Got {} partitions from these topics: {}", partitions.size(), topics);
 
 		if (LOG.isInfoEnabled()) {
-			logPartitionInfo(partitionInfos);
+			logPartitionInfo(LOG, partitions);
 		}
 
-		this.consumerId = UUID.randomUUID().toString();
+		// register these partitions
+		setSubscribedPartitions(partitions);
+	}
+
+	@Override
+	protected AbstractFetcher<T, ?> createFetcher(
+			SourceContext<T> sourceContext,
+			List<KafkaTopicPartition> thisSubtaskPartitions,
+			SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+			SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+			StreamingRuntimeContext runtimeContext) throws Exception {
+
+		boolean useMetrics = !Boolean.valueOf(properties.getProperty(KEY_DISABLE_METRICS, "false"));
+
+		return new Kafka09Fetcher<>(sourceContext, thisSubtaskPartitions,
+				watermarksPeriodic, watermarksPunctuated,
+				runtimeContext, deserializer,
+				properties, pollTimeout, useMetrics);
+		
 	}
 
+	// ------------------------------------------------------------------------
+	//  Utilities 
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Converts a list of Kafka PartitionInfo's to Flink's KafkaTopicPartition (which are serializable)
+	 * 
 	 * @param partitions A list of Kafka PartitionInfos.
 	 * @return A list of KafkaTopicPartitions
 	 */
-	public static List<KafkaTopicPartition> convertToFlinkKafkaTopicPartition(List<PartitionInfo> partitions) {
-		requireNonNull(partitions, "The given list of partitions was null");
+	private static List<KafkaTopicPartition> convertToFlinkKafkaTopicPartition(List<PartitionInfo> partitions) {
+		checkNotNull(partitions);
+
 		List<KafkaTopicPartition> ret = new ArrayList<>(partitions.size());
-		for(PartitionInfo pi: partitions) {
+		for (PartitionInfo pi : partitions) {
 			ret.add(new KafkaTopicPartition(pi.topic(), pi.partition()));
 		}
 		return ret;
 	}
 
-	public static List<TopicPartition> convertToKafkaTopicPartition(List<KafkaTopicPartition> partitions) {
-		List<TopicPartition> ret = new ArrayList<>(partitions.size());
-		for(KafkaTopicPartition ktp: partitions) {
-			ret.add(new TopicPartition(ktp.getTopic(), ktp.getPartition()));
-		}
-		return ret;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Source life cycle
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void open(Configuration parameters) throws Exception {
-		super.open(parameters);
-
-		final int numConsumers = getRuntimeContext().getNumberOfParallelSubtasks();
-		final int thisConsumerIndex = getRuntimeContext().getIndexOfThisSubtask();
-
-		// pick which partitions we work on
-		this.subscribedPartitionsAsFlink = assignPartitions(this.partitionInfos, numConsumers, thisConsumerIndex);
-		if(this.subscribedPartitionsAsFlink.isEmpty()) {
-			LOG.info("This consumer doesn't have any partitions assigned");
-			this.partitionState = null;
-			return;
-		} else {
-			StreamingRuntimeContext streamingRuntimeContext = (StreamingRuntimeContext) getRuntimeContext();
-			// if checkpointing is enabled, we are not automatically committing to Kafka.
-			properties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(!streamingRuntimeContext.isCheckpointingEnabled()));
-			this.consumer = new KafkaConsumer<>(properties);
-		}
-		subscribedPartitions = convertToKafkaTopicPartition(subscribedPartitionsAsFlink);
-
-		this.consumer.assign(this.subscribedPartitions);
-
-		// register Kafka metrics to Flink accumulators
-		if(!Boolean.valueOf(properties.getProperty(KEY_DISABLE_METRICS, "false"))) {
-			Map<MetricName, ? extends Metric> metrics = this.consumer.metrics();
-			if(metrics == null) {
-				// MapR's Kafka implementation returns null here.
-				LOG.info("Consumer implementation does not support metrics");
-			} else {
-				for (Map.Entry<MetricName, ? extends Metric> metric : metrics.entrySet()) {
-					String name = consumerId + "-consumer-" + metric.getKey().name();
-					DefaultKafkaMetricAccumulator kafkaAccumulator = DefaultKafkaMetricAccumulator.createFor(metric.getValue());
-					// best effort: we only add the accumulator if available.
-					if (kafkaAccumulator != null) {
-						getRuntimeContext().addAccumulator(name, kafkaAccumulator);
-					}
-				}
-			}
-		}
-
-		// check if we need to explicitly seek to a specific offset (restore case)
-		if(restoreToOffset != null) {
-			// we are in a recovery scenario
-			for(Map.Entry<KafkaTopicPartition, Long> info: restoreToOffset.entrySet()) {
-				// seek all offsets to the right position
-				this.consumer.seek(new TopicPartition(info.getKey().getTopic(), info.getKey().getPartition()), info.getValue() + 1);
-			}
-			this.partitionState = restoreInfoFromCheckpoint();
-		} else {
-			this.partitionState = new HashMap<>();
-		}
-	}
-
-
-
-	@Override
-	public void run(SourceContext<T> sourceContext) throws Exception {
-		if(consumer != null) {
-			consumerThread = new ConsumerThread<>(this, sourceContext);
-			consumerThread.setDaemon(true);
-			consumerThread.start();
-			// wait for the consumer to stop
-			while(consumerThread.isAlive()) {
-				if(consumerThreadException != null) {
-					throw new RuntimeException("ConsumerThread threw an exception: " + consumerThreadException.getMessage(), consumerThreadException);
-				}
-				try {
-					consumerThread.join(50);
-				} catch (InterruptedException ie) {
-					consumerThread.shutdown();
-				}
-			}
-			// check again for an exception
-			if(consumerThreadException != null) {
-				throw new RuntimeException("ConsumerThread threw an exception: " + consumerThreadException.getMessage(), consumerThreadException);
-			}
-		} else {
-			// this source never completes, so emit a Long.MAX_VALUE watermark
-			// to not block watermark forwarding
-			sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE));
-
-			final Object waitLock = new Object();
-			this.waitThread = Thread.currentThread();
-			while (running) {
-				// wait until we are canceled
-				try {
-					//noinspection SynchronizationOnLocalVariableOrMethodParameter
-					synchronized (waitLock) {
-						waitLock.wait();
-					}
-				}
-				catch (InterruptedException e) {
-					// do nothing, check our "running" status
-				}
-			}
-		}
-		// close the context after the work was done. this can actually only
-		// happen when the fetcher decides to stop fetching
-		sourceContext.close();
-	}
-
-	@Override
-	public void cancel() {
-		// set ourselves as not running
-		running = false;
-		if(this.consumerThread != null) {
-			this.consumerThread.shutdown();
-		} else {
-			// the consumer thread is not running, so we have to interrupt our own thread
-			if(waitThread != null) {
-				waitThread.interrupt();
-			}
-		}
-	}
-
-	@Override
-	public void close() throws Exception {
-		cancel();
-		super.close();
-	}
-
-	// ------------------------------------------------------------------------
-	//  Checkpoint and restore
-	// ------------------------------------------------------------------------
-
-
-	@Override
-	protected void commitOffsets(HashMap<KafkaTopicPartition, Long> checkpointOffsets) {
-		if(!running) {
-			LOG.warn("Unable to commit offsets on closed consumer");
-			return;
-		}
-		Map<TopicPartition, OffsetAndMetadata> kafkaCheckpointOffsets = convertToCommitMap(checkpointOffsets);
-		synchronized (this.consumer) {
-			this.consumer.commitSync(kafkaCheckpointOffsets);
-		}
-	}
-
-	public static Map<TopicPartition, OffsetAndMetadata> convertToCommitMap(HashMap<KafkaTopicPartition, Long> checkpointOffsets) {
-		Map<TopicPartition, OffsetAndMetadata> ret = new HashMap<>(checkpointOffsets.size());
-		for(Map.Entry<KafkaTopicPartition, Long> partitionOffset: checkpointOffsets.entrySet()) {
-			ret.put(new TopicPartition(partitionOffset.getKey().getTopic(), partitionOffset.getKey().getPartition()),
-					new OffsetAndMetadata(partitionOffset.getValue(), ""));
-		}
-		return ret;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Miscellaneous utilities 
-	// ------------------------------------------------------------------------
-
-
-	protected static void setDeserializer(Properties props) {
-		if (!props.contains(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) {
-			props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getCanonicalName());
-		} else {
-			LOG.warn("Overwriting the '{}' is not recommended", ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
-		}
-
-		if (!props.contains(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) {
-			props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getCanonicalName());
-		} else {
-			LOG.warn("Overwriting the '{}' is not recommended", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
-		}
-	}
-
 	/**
-	 * We use a separate thread for executing the KafkaConsumer.poll(timeout) call because Kafka is not
-	 * handling interrupts properly. On an interrupt (which happens automatically by Flink if the task
-	 * doesn't react to cancel() calls), the poll() method might never return.
-	 * On cancel, we'll wakeup the .poll() call and wait for it to return
+	 * Makes sure that the ByteArrayDeserializer is registered in the Kafka properties.
+	 * 
+	 * @param props The Kafka properties to register the serializer in.
 	 */
-	private static class ConsumerThread<T> extends Thread {
-		private final FlinkKafkaConsumer09<T> flinkKafkaConsumer;
-		private final SourceContext<T> sourceContext;
-		private boolean running = true;
-
-		public ConsumerThread(FlinkKafkaConsumer09<T> flinkKafkaConsumer, SourceContext<T> sourceContext) {
-			this.flinkKafkaConsumer = flinkKafkaConsumer;
-			this.sourceContext = sourceContext;
-		}
+	private static void setDeserializer(Properties props) {
+		final String deSerName = ByteArrayDeserializer.class.getCanonicalName();
 
-		@Override
-		public void run() {
-			try {
-				long pollTimeout = Long.parseLong(flinkKafkaConsumer.properties.getProperty(KEY_POLL_TIMEOUT, Long.toString(DEFAULT_POLL_TIMEOUT)));
-				pollLoop: while (running) {
-					ConsumerRecords<byte[], byte[]> records;
-					//noinspection SynchronizeOnNonFinalField
-					synchronized (flinkKafkaConsumer.consumer) {
-						try {
-							records = flinkKafkaConsumer.consumer.poll(pollTimeout);
-						} catch (WakeupException we) {
-							if (running) {
-								throw we;
-							}
-							// leave loop
-							continue;
-						}
-					}
-					// get the records for each topic partition
-					for (int i = 0; i < flinkKafkaConsumer.subscribedPartitions.size(); i++) {
-						TopicPartition partition = flinkKafkaConsumer.subscribedPartitions.get(i);
-						KafkaTopicPartition flinkPartition = flinkKafkaConsumer.subscribedPartitionsAsFlink.get(i);
-						List<ConsumerRecord<byte[], byte[]>> partitionRecords = records.records(partition);
-						//noinspection ForLoopReplaceableByForEach
-						for (int j = 0; j < partitionRecords.size(); j++) {
-							ConsumerRecord<byte[], byte[]> record = partitionRecords.get(j);
-							T value = flinkKafkaConsumer.deserializer.deserialize(record.key(), record.value(), record.topic(), record.partition(),record.offset());
-							if(flinkKafkaConsumer.deserializer.isEndOfStream(value)) {
-								// end of stream signaled
-								running = false;
-								break pollLoop;
-							}
-							synchronized (sourceContext.getCheckpointLock()) {
-								flinkKafkaConsumer.processElement(sourceContext, flinkPartition, value, record.offset());
-							}
-						}
-					}
-				}
-			} catch(Throwable t) {
-				if(running) {
-					this.flinkKafkaConsumer.stopWithError(t);
-				} else {
-					LOG.debug("Stopped ConsumerThread threw exception", t);
-				}
-			} finally {
-				try {
-					flinkKafkaConsumer.consumer.close();
-				} catch(Throwable t) {
-					LOG.warn("Error while closing consumer", t);
-				}
-			}
-		}
+		Object keyDeSer = props.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
+		Object valDeSer = props.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
 
-		/**
-		 * Try to shutdown the thread
-		 */
-		public void shutdown() {
-			this.running = false;
-			this.flinkKafkaConsumer.consumer.wakeup();
+		if (keyDeSer != null && !keyDeSer.equals(deSerName)) {
+			LOG.warn("Ignoring configured key DeSerializer ({})", ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
+		}
+		if (valDeSer != null && !valDeSer.equals(deSerName)) {
+			LOG.warn("Ignoring configured value DeSerializer ({})", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
 		}
-	}
 
-	private void stopWithError(Throwable t) {
-		this.consumerThreadException = t;
+		props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deSerName);
+		props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deSerName);
 	}
 }


[03/14] flink git commit: [FLINK-3716] [kafka consumer] Decreasing socket timeout so testFailOnNoBroker() will pass before JUnit timeout

Posted by se...@apache.org.
[FLINK-3716] [kafka consumer] Decreasing socket timeout so testFailOnNoBroker() will pass before JUnit timeout

This closes #1864


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

Branch: refs/heads/master
Commit: af799883371b7a78c48f227b25b7a0b099dabf93
Parents: 3c93103
Author: Todd Lisonbee <to...@intel.com>
Authored: Thu Apr 7 17:47:19 2016 -0700
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Apr 13 01:10:54 2016 +0200

----------------------------------------------------------------------
 .../flink/streaming/connectors/kafka/KafkaConsumerTestBase.java     | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/af799883/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index aa5344b..dd468a4 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -152,6 +152,7 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBase {
 			properties.setProperty("zookeeper.connect", "localhost:80");
 			properties.setProperty("group.id", "test");
 			properties.setProperty("request.timeout.ms", "3000"); // let the test fail fast
+			properties.setProperty("socket.timeout.ms", "3000");
 			properties.setProperty("session.timeout.ms", "2000");
 			properties.setProperty("fetch.max.wait.ms", "2000");
 			properties.setProperty("heartbeat.interval.ms", "1000");